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/25 06:10:47 UTC

[GitHub] [flink] azagrebin opened a new pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

azagrebin opened a new pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682
 
 
   ## What is the purpose of the change
   
   #10161 introduced canceling and waiting of all tasks in TM to release resources and exit before shutting down TM services on stop. #10034 assigned a separate memory manager per slot and #10330 introduced dynamic slots and state to `TaskSlotTable`. To achieve clean lifecycle of slot, table and TM, this PR introduces freeing of all slots before shutting down TM services and stopping slot table on TM stop. Freeing of all slots cancels the running tasks and releases slot resources, including memory manager.
    
   ## Brief change log
   
     - Introduce closing future to `TaskSlot`
     - Introduce `TaskSlotTable#freeAllSlots`
     - Adjust task executor tests
     - Add more test coverage for `TaskSlot` and `TaskSlotTable`
   
   ## Verifying this change
   
   CI unit tests
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 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] tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362823684
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java
 ##########
 @@ -57,6 +61,7 @@ public TaskManagerServicesBuilder() {
 		broadcastVariableManager = new BroadcastVariableManager();
 		taskEventDispatcher = new TaskEventDispatcher();
 		taskSlotTable = mock(TaskSlotTable.class);
+		when(taskSlotTable.freeAllSlots(any(Throwable.class))).thenReturn(CompletableFuture.completedFuture(null));
 
 Review comment:
   Instead of adding mockito logic, I would suggest to extract an interface and provide a proper testing implementation.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 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] tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362847991
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
 ##########
 @@ -731,70 +697,6 @@ public void invoke() throws Exception {
 		}
 	}
 
-	@Test
-	public void testTaskInterruptionAndTerminationOnShutdown() throws Exception {
 
 Review comment:
   By removing this test we lose coverage that we don't stop the `TaskExecutor` before all tasks have been terminated. I couldn't find another test testing this scenario. Hence, please add one or revert the removal.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369059793
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private CompletableFuture<Void> closingFuture;
+
+	/** {@link Executor} to schedule internal calls to the main thread. */
+	private Executor mainThreadExecutor;
+
+	public TaskSlotTableImpl(
+		final int numberSlots,
+		final ResourceProfile totalAvailableResourceProfile,
+		final ResourceProfile defaultSlotResourceProfile,
+		final int memoryPageSize,
+		final TimerService<AllocationID> timerService) {
+
+		Preconditions.checkArgument(0 < numberSlots, "The number of task slots must be greater than 0.");
+
+		this.numberSlots = numberSlots;
+		this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+		this.memoryPageSize = memoryPageSize;
+
+		this.taskSlots = new HashMap<>(numberSlots);
+
+		this.timerService = Preconditions.checkNotNull(timerService);
+
+		budgetManager = new ResourceBudgetManager(Preconditions.checkNotNull(totalAvailableResourceProfile));
+
+		allocatedSlots = new HashMap<>(numberSlots);
+
+		taskSlotMappings = new HashMap<>(4 * numberSlots);
+
+		slotsPerJob = new HashMap<>(4);
+
+		slotActions = null;
+		state = State.CREATED;
+	}
+
+	@Override
+	public void start(SlotActions initialSlotActions, Executor mainThreadExecutor) {
+		Preconditions.checkState(
+			state == State.CREATED,
+			"The %s has to be just created before starting",
+			TaskSlotTableImpl.class.getSimpleName());
+		this.slotActions = Preconditions.checkNotNull(initialSlotActions);
+		this.mainThreadExecutor = Preconditions.checkNotNull(mainThreadExecutor);
+
+		timerService.start(this);
+
+		state = State.RUNNING;
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		if (state == State.CREATED) {
+			state = State.CLOSED;
+			closingFuture = CompletableFuture.completedFuture(null);
+		} else if (state == State.RUNNING) {
+			state = State.CLOSING;
+			final FlinkException cause = new FlinkException("Closing task slot table");
+			closingFuture = FutureUtils
+				.waitForAll(
+					new ArrayList<>(allocatedSlots.values())
+						.stream()
+						.map(slot -> freeSlot(slot, cause))
+						.collect(Collectors.toList()))
+				.thenRunAsync(
+					() -> {
+						state = State.CLOSED;
+						timerService.stop();
 
 Review comment:
   Given that this now runs in the main thread, we could also clean up the internal structures.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362840043
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTest.java
 ##########
 @@ -0,0 +1,52 @@
+package org.apache.flink.runtime.taskexecutor.slot;/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.junit.Test;
+
+import java.util.concurrent.CompletableFuture;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class TaskSlotTest {
 
 Review comment:
   We should also test that the `MemoryManager` stays alive until all `Tasks` have terminated.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369052941
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -238,8 +208,112 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 				is(new SlotStatus(new SlotID(resourceId, 1), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, null, null)),
 				is(new SlotStatus(new SlotID(resourceId, 2), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, null, null)),
 				is(new SlotStatus(SlotID.generateDynamicSlotID(resourceId), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, jobId, allocationId3))));
-		} finally {
-			taskSlotTable.stop();
 		}
 	}
+
+	@Test
+	public void testAllocateSlot() throws Exception {
+		final JobID jobId = new JobID();
+		final AllocationID allocationId = new AllocationID();
+		try (final TaskSlotTable taskSlotTable =
+				 createTaskSlotTableWithAllocatedSlot(jobId, allocationId, new TestingSlotActionsBuilder().build())) {
+			Iterator<TaskSlot> allocatedSlots = taskSlotTable.getAllocatedSlots(jobId);
+			TaskSlot nextSlot = allocatedSlots.next();
+			assertThat(nextSlot.getIndex(), is(0));
+			assertThat(nextSlot.getAllocationId(), is(allocationId));
+			assertThat(nextSlot.getJobId(), is(jobId));
+			assertThat(allocatedSlots.hasNext(), is(false));
+		}
+	}
+
+	@Test
+	public void testAddTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = TaskSlotUtils.createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		try (final TaskSlotTable taskSlotTable =
+				 createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build())) {
+			Iterator<Task> tasks = taskSlotTable.getTasks(jobId);
+			Task nextTask = tasks.next();
+			assertThat(nextTask.getExecutionId(), is(executionAttemptId));
+			assertThat(nextTask.getAllocationId(), is(allocationId));
+			assertThat(tasks.hasNext(), is(false));
+		}
+	}
+
+	@Test(timeout = 10000)
+	public void testRemoveTaskCallsFreeSlotAction() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		CompletableFuture<AllocationID> freeSlotFuture = new CompletableFuture<>();
+		SlotActions slotActions = new TestingSlotActions(freeSlotFuture::complete, (aid, uid) -> {});
+		Task task = TaskSlotUtils.createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		try (final TaskSlotTable taskSlotTable = createTaskSlotTableWithStartedTask(task, slotActions)) {
+			// we have to initiate closing of the slot externally
+			// to enable that the last remaining finished task does the final slot freeing
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
+			assertThat(freeSlotFuture.get(), is(allocationId));
+		}
+	}
+
+	@Test(timeout = 10000)
+	public void testFreeSlotInterruptsSubmittedTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		TestInterruptableInvokable.reset();
+		Task task = TaskSlotUtils.createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			TestInterruptableInvokable.class);
+		final TaskSlotTable taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+		TestInterruptableInvokable.waitForStarted();
+		assertThat(taskSlotTable.freeSlot(allocationId), is(-1));
+		TestInterruptableInvokable.waitForInterrupted();
+		CompletableFuture<Void> closingFuture = taskSlotTable.closeAsync();
+		assertThat(closingFuture.isDone(), is(false));
+		TestInterruptableInvokable.finish();
+		closingFuture.get();
 
 Review comment:
   nit: technically we are testing two things here. Don't have to change this now but keep this in mind for the future.

----------------------------------------------------------------
This is an automated message from the 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 commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 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] tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362826149
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTest.java
 ##########
 @@ -0,0 +1,52 @@
+package org.apache.flink.runtime.taskexecutor.slot;/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.junit.Test;
+
+import java.util.concurrent.CompletableFuture;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class TaskSlotTest {
+	@Test
 
 Review comment:
   I think a line break is missing before the annotation.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369054694
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java
 ##########
 @@ -73,4 +79,30 @@ public static ResourceProfile createTotalResourceProfile(int numberOfSlots) {
 	public static TimerService<AllocationID> createDefaultTimerService(long shutdownTimeout) {
 		return new TimerService<>(TestingUtils.defaultExecutor(), shutdownTimeout);
 	}
+
+	public static Task createTask1(
+		JobID jobId,
+		ExecutionAttemptID executionAttemptId,
+		AllocationID allocationId,
+		Class<? extends AbstractInvokable> invokable) throws Exception {
 
 Review comment:
   nit: the formatting is off wrt `createTask`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362836806
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -239,7 +239,12 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 				is(new SlotStatus(new SlotID(resourceId, 2), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, null, null)),
 				is(new SlotStatus(SlotID.generateDynamicSlotID(resourceId), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, jobId, allocationId3))));
 		} finally {
-			taskSlotTable.stop();
+			stopTable(taskSlotTable);
 		}
 	}
+
+	private static void stopTable(TaskSlotTable taskSlotTable) {
+		taskSlotTable.freeAllSlots(null).join();
+		taskSlotTable.stop();
+	}
 
 Review comment:
   This is a good example for why the current API design is not very elegant. Stopping a `TaskSlotTable` should simply be `taskSlotTable.stop`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362825818
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTest.java
 ##########
 @@ -0,0 +1,52 @@
+package org.apache.flink.runtime.taskexecutor.slot;/*
 
 Review comment:
   This does not look right.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362841198
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -243,8 +252,181 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 		}
 	}
 
+	@Test
+	public void testAllocateSlot() {
+		final JobID jobId = new JobID();
+		final AllocationID allocationId = new AllocationID();
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithAllocatedSlot(
+				jobId,
+				allocationId,
+				new TestingSlotActionsBuilder().build());
+			Iterator<TaskSlot> allocatedSlots = taskSlotTable.getAllocatedSlots(jobId);
+			TaskSlot nextSlot = allocatedSlots.next();
+			assertThat(nextSlot.getIndex(), is(0));
+			assertThat(nextSlot.getAllocationId(), is(allocationId));
+			assertThat(nextSlot.getJobId(), is(jobId));
+			assertThat(allocatedSlots.hasNext(), is(false));
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test
+	public void testAddTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+			Iterator<Task> tasks = taskSlotTable.getTasks(jobId);
+			Task nextTask = tasks.next();
+			assertThat(nextTask.getExecutionId(), is(executionAttemptId));
+			assertThat(nextTask.getAllocationId(), is(allocationId));
+			assertThat(tasks.hasNext(), is(false));
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test(timeout = 10000)
+	public void testRemoveTaskCallsFreeSlotAction() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		CompletableFuture<AllocationID> freeSlotFuture = new CompletableFuture<>();
+		SlotActions slotActions = new TestingSlotActions(freeSlotFuture::complete, (aid, uid) -> {});
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, slotActions);
+			taskSlotTable.freeSlot(allocationId);
 
 Review comment:
   Why do we need to call this method for the test?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362839773
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTest.java
 ##########
 @@ -0,0 +1,52 @@
+package org.apache.flink.runtime.taskexecutor.slot;/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.junit.Test;
+
+import java.util.concurrent.CompletableFuture;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class TaskSlotTest {
 
 Review comment:
   I think this test class lacks some important tests: Test that the `TaskSlot` is not closed before all contained tasks have terminated. 
   
   Moreover, tests should be in the commit where the feature is being introduced.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369045071
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
 ##########
 @@ -2044,6 +1907,97 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa
 			TaskManagerRunner.createBackPressureSampleService(configuration, rpc.getScheduledExecutor()));
 	}
 
+	private TaskExecutorTestingContext createTaskExecutorTestingContext(int numberOfSlots) {
+		return createTaskExecutorTestingContext(TaskSlotUtils.createTaskSlotTable(numberOfSlots));
+	}
+
+	private TaskExecutorTestingContext createTaskExecutorTestingContext(final TaskSlotTable taskSlotTable) {
+		final OneShotLatch offerSlotsLatch = new OneShotLatch();
+		final TestingJobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder()
+			.setOfferSlotsFunction((resourceID, slotOffers) -> {
+				offerSlotsLatch.trigger();
+				return CompletableFuture.completedFuture(slotOffers);
+			}).build();
+		final JobLeaderService jobLeaderService = new JobLeaderService(
+			taskManagerLocation,
+			RetryingRegistrationConfiguration.defaultConfiguration());
+		TaskExecutorLocalStateStoresManager stateStoresManager = null;
+		try {
+			stateStoresManager = createTaskExecutorLocalStateStoresManager();
+		} catch (IOException e) {
+			ExceptionUtils.rethrow(e);
+		}
 
 Review comment:
   Why do we need to rethrow exceptions as `RuntimeException`? Couldn't we allow this method to throw an exception?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362823661
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java
 ##########
 @@ -124,6 +124,7 @@ private TaskSubmissionTestEnvironment(
 			this.taskSlotTable = mock(TaskSlotTable.class);
 			when(taskSlotTable.tryMarkSlotActive(eq(jobId), any())).thenReturn(true);
 			when(taskSlotTable.addTask(any(Task.class))).thenReturn(true);
+			when(taskSlotTable.freeAllSlots(any(Throwable.class))).thenReturn(CompletableFuture.completedFuture(null));
 
 Review comment:
   Instead of adding mockito logic, I would suggest to extract an interface and provide a proper testing implementation.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 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] azagrebin commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568846213
 
 
   cc @xintongsong 

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368065079
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -462,7 +471,7 @@ public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFo
 	 * @return True if the timeout is valid; otherwise false
 	 */
 	public boolean isValidTimeout(AllocationID allocationId, UUID ticket) {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   Same here, this might allow a race condition between this method and the stopping of the `TimerService`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369054907
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java
 ##########
 @@ -73,4 +79,30 @@ public static ResourceProfile createTotalResourceProfile(int numberOfSlots) {
 	public static TimerService<AllocationID> createDefaultTimerService(long shutdownTimeout) {
 		return new TimerService<>(TestingUtils.defaultExecutor(), shutdownTimeout);
 	}
+
+	public static Task createTask1(
 
 Review comment:
   Is this duplicating `createTask`?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145789064 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145909543 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145789064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587) 
   * 8c7c178603caf99a23ed9b92f07527d8e5f7169c Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145909543) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597) 
   
   <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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369155082
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java
 ##########
 @@ -73,4 +79,30 @@ public static ResourceProfile createTotalResourceProfile(int numberOfSlots) {
 	public static TimerService<AllocationID> createDefaultTimerService(long shutdownTimeout) {
 		return new TimerService<>(TestingUtils.defaultExecutor(), shutdownTimeout);
 	}
+
+	public static Task createTask1(
 
 Review comment:
   unused leftover

----------------------------------------------------------------
This is an automated message from the 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] AHeise commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
AHeise commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-576580269
 
 
   > I think the safest way to handle timer service concurrency issues is to schedule its stopping to the main thread of task executor. Then there is only a thread-confined access to its internal container for registered callbacks.
   Would be awesome if you could use that and simplify the threading model.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369048416
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/TestInterruptableInvokable.java
 ##########
 @@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testtasks;
+
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.util.ExceptionUtils;
+
+/**
+ * Test invokable which completes the given future when interrupted (can be used only once).
 
 Review comment:
   ```suggestion
    * Test invokable which triggers the given latches when interrupted (can be used only once).
   ```

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369047483
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/StubTaskSlotTable.java
 ##########
 @@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskmanager.Task;
+
+import javax.annotation.Nullable;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Stub implementation of {@link TaskSlotTable} for tests.
+ */
+public class StubTaskSlotTable implements TaskSlotTable {
 
 Review comment:
   I think so far the convention was to name these kind of implementations `TestingTaskSlotTable`.

----------------------------------------------------------------
This is an automated message from the 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] AHeise edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
AHeise edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-576580269
 
 
   > I think the safest way to handle timer service concurrency issues is to schedule its stopping to the main thread of task executor. Then there is only a thread-confined access to its internal container for registered callbacks.
   
   Would be awesome if you could use that and simplify the threading model.

----------------------------------------------------------------
This is an automated message from the 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] azagrebin commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-573687637
 
 
   Thanks for the review @tillrohrmann, I addressed comments.

----------------------------------------------------------------
This is an automated message from the 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r365830481
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -376,7 +372,7 @@ private void handleStartTaskExecutorServicesException(Exception e) throws Except
 
 		return FutureUtils
 			.runAfterwards(
-				taskCompletionTracker.failIncompleteTasksAndGetTerminationFuture(),
+				taskSlotTable.freeAllSlots(new FlinkException("The task executor is shutting down.")),
 
 Review comment:
   Indeed, it is complicated. I tried to avoid changing too much the process of freeing slots. At the moment it involves quite some steps:
   
   - JM calls freeSlot and puts the slot into the releasing state
   - task update state final (TE)
   - remove task (table)
   - if slot is in releasing state and it has no running tasks
     - the async table.freeSlot and
     - notifies RM/close JM connection (slot actions impl in TE)
   
   It would be nice that `table.freeSlot` would not have to go through TE but I guess somehow `table.freeSlot and notifies RM/close JM connection` has to be scheduled async into the main thread atomically.
   
   True, it would be nicer if task management was part of the slot as task belongs to it then it could be tested only there and slot freeing/closing would guarantee interruption, now it is mixed into the table class.
   
   The first approach in this PR also kept slot/table not dealing with threading and futures leaving TM taking care of it. I agree that the suggested approach separates better responsibilities w/o changing the existing process of freeing slot too much and introducing async behaviour to slot/table should not be a big problem now.
   
   I implemented the suggested approach with `closeAsync`. Some final cleanup of slot/table (timeService.stop and  memoryManager.shutdown) happens in sync callbacks to the termination of tasks w/o scheduling it to the main thread of TM. Also, the final cleanup of table internal data structures after slot freeing potentially happens after the closing future but better to be scheduled before. Although this does not look implicitly a problem now, it is better to rethink more the design and responsibilities by the next opportunity and keep all actions thread-confined to avoid accidental putting of critical stuff into the callbacks w/o proper synchronisation. As discussed, one solution could be that slot/table get main thread executor in future and become more independent from TM.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370156250
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##########
 @@ -478,10 +483,6 @@ public boolean isBackPressured() {
 	//  Task Execution
 	// ------------------------------------------------------------------------
 
-	/**
-	 * Returns the current execution state of the task.
-	 * @return The current execution state of the task.
-	 */
 
 Review comment:
   Why did you remove the JavaDocs?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369053860
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -238,8 +208,112 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 				is(new SlotStatus(new SlotID(resourceId, 1), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, null, null)),
 				is(new SlotStatus(new SlotID(resourceId, 2), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, null, null)),
 				is(new SlotStatus(SlotID.generateDynamicSlotID(resourceId), TaskSlotUtils.DEFAULT_RESOURCE_PROFILE, jobId, allocationId3))));
-		} finally {
-			taskSlotTable.stop();
 		}
 	}
+
+	@Test
+	public void testAllocateSlot() throws Exception {
+		final JobID jobId = new JobID();
+		final AllocationID allocationId = new AllocationID();
+		try (final TaskSlotTable taskSlotTable =
+				 createTaskSlotTableWithAllocatedSlot(jobId, allocationId, new TestingSlotActionsBuilder().build())) {
+			Iterator<TaskSlot> allocatedSlots = taskSlotTable.getAllocatedSlots(jobId);
+			TaskSlot nextSlot = allocatedSlots.next();
+			assertThat(nextSlot.getIndex(), is(0));
+			assertThat(nextSlot.getAllocationId(), is(allocationId));
+			assertThat(nextSlot.getJobId(), is(jobId));
+			assertThat(allocatedSlots.hasNext(), is(false));
+		}
+	}
+
+	@Test
+	public void testAddTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = TaskSlotUtils.createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		try (final TaskSlotTable taskSlotTable =
+				 createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build())) {
+			Iterator<Task> tasks = taskSlotTable.getTasks(jobId);
+			Task nextTask = tasks.next();
+			assertThat(nextTask.getExecutionId(), is(executionAttemptId));
+			assertThat(nextTask.getAllocationId(), is(allocationId));
+			assertThat(tasks.hasNext(), is(false));
+		}
+	}
+
+	@Test(timeout = 10000)
+	public void testRemoveTaskCallsFreeSlotAction() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		CompletableFuture<AllocationID> freeSlotFuture = new CompletableFuture<>();
+		SlotActions slotActions = new TestingSlotActions(freeSlotFuture::complete, (aid, uid) -> {});
+		Task task = TaskSlotUtils.createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		try (final TaskSlotTable taskSlotTable = createTaskSlotTableWithStartedTask(task, slotActions)) {
+			// we have to initiate closing of the slot externally
+			// to enable that the last remaining finished task does the final slot freeing
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
+			assertThat(freeSlotFuture.get(), is(allocationId));
+		}
+	}
+
+	@Test(timeout = 10000)
+	public void testFreeSlotInterruptsSubmittedTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		TestInterruptableInvokable.reset();
+		Task task = TaskSlotUtils.createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			TestInterruptableInvokable.class);
+		final TaskSlotTable taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+		TestInterruptableInvokable.waitForStarted();
+		assertThat(taskSlotTable.freeSlot(allocationId), is(-1));
+		TestInterruptableInvokable.waitForInterrupted();
+		CompletableFuture<Void> closingFuture = taskSlotTable.closeAsync();
+		assertThat(closingFuture.isDone(), is(false));
+		TestInterruptableInvokable.finish();
+		closingFuture.get();
+	}
+
+	private static TaskSlotTableImpl createTaskSlotTableAndStart(int numberOfSlots) {
+		final TaskSlotTableImpl taskSlotTable = TaskSlotUtils.createTaskSlotTable(numberOfSlots);
+		taskSlotTable.start(new TestingSlotActionsBuilder().build(), Executors.directExecutor());
 
 Review comment:
   Using the `directExecutor` can be problematic as it allows for thread interleavings which are not possible with a proper `Executor`. I assume that in this case it is not problematic, though, because we use the executor only for the clean up call.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362825084
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##########
 @@ -157,6 +159,9 @@ public void testTaskSubmission() throws Exception {
 			tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get();
 
 			taskRunningFuture.get();
+			taskFinishedFuture.get();
+
+			taskSlotTable.removeTask(tdd.getExecutionAttemptId());
 
 Review comment:
   Why do we have to explicitly call `removeTask`? This seems to be quite cumbersome.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370154316
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
 ##########
 @@ -54,7 +53,7 @@
  * <p>An allocated or active slot can only be freed if it is empty. If it is not empty, then it's state
  * can be set to releasing indicating that it can be freed once it becomes empty.
 
 Review comment:
   nit: Description of `T` is missing.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362829356
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -146,12 +149,7 @@ public void start(SlotActions initialSlotActions) {
 	public void stop() {
 		started = false;
 		timerService.stop();
-		allocatedSlots
-			.values()
-			.stream()
-			.filter(slot -> !taskSlots.containsKey(slot.getIndex()))
-			.forEach(TaskSlot::close);
-		allocatedSlots.clear();
+		Preconditions.checkState(allocatedSlots.isEmpty(), "All slots have to be freed before stopping TaskSlotTable");
 		taskSlots.values().forEach(TaskSlot::close);
 
 Review comment:
   This looks a bit odd since we checked the line above that `allocatedSlots` is empty.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369018626
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -397,62 +135,7 @@ public int freeSlot(AllocationID allocationId) throws SlotNotFoundException {
 	 * @throws SlotNotFoundException if there is not task slot for the given allocation id
 	 * @return The freed TaskSlot. If the TaskSlot cannot be freed then null.
 
 Review comment:
   The return value description seems to be wrong.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145789064 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145909543 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   Hash:c545a97ce8fb7718fdf24c724d49ff5293f72364 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:c545a97ce8fb7718fdf24c724d49ff5293f72364
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145789064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587) 
   * 8c7c178603caf99a23ed9b92f07527d8e5f7169c Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145909543) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597) 
   * c545a97ce8fb7718fdf24c724d49ff5293f72364 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r365833036
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -243,8 +252,181 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 		}
 	}
 
+	@Test
+	public void testAllocateSlot() {
+		final JobID jobId = new JobID();
+		final AllocationID allocationId = new AllocationID();
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithAllocatedSlot(
+				jobId,
+				allocationId,
+				new TestingSlotActionsBuilder().build());
+			Iterator<TaskSlot> allocatedSlots = taskSlotTable.getAllocatedSlots(jobId);
+			TaskSlot nextSlot = allocatedSlots.next();
+			assertThat(nextSlot.getIndex(), is(0));
+			assertThat(nextSlot.getAllocationId(), is(allocationId));
+			assertThat(nextSlot.getJobId(), is(jobId));
+			assertThat(allocatedSlots.hasNext(), is(false));
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test
+	public void testAddTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+			Iterator<Task> tasks = taskSlotTable.getTasks(jobId);
+			Task nextTask = tasks.next();
+			assertThat(nextTask.getExecutionId(), is(executionAttemptId));
+			assertThat(nextTask.getAllocationId(), is(allocationId));
+			assertThat(tasks.hasNext(), is(false));
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
 
 Review comment:
   I removed the unnecessary calls, also for subsequent comments, after the suggested refactoring and added comment in code for one where I think it was needed.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369014847
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private CompletableFuture<Void> closingFuture;
 
 Review comment:
   I would suggest to initialise this field in the constructor. That way we would not have to deal with uninitialised fields.

----------------------------------------------------------------
This is an automated message from the 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r365811773
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##########
 @@ -157,6 +159,9 @@ public void testTaskSubmission() throws Exception {
 			tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get();
 
 			taskRunningFuture.get();
+			taskFinishedFuture.get();
+
+			taskSlotTable.removeTask(tdd.getExecutionAttemptId());
 
 Review comment:
   It was checked in stop method that there is no tasks. Now it is not a problem with new closing approach.

----------------------------------------------------------------
This is an automated message from the 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 commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568846548
 
 
   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 a7411a5d1a86381078670898c1f20be4fe3a88f4 (Wed Dec 25 06:13:16 UTC 2019)
   
   **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.
 
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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369049298
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/EmptyInvokable.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testtasks;
+
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+/**
+ * Test invokable which does nothing and immediately completes.
+ */
+public class EmptyInvokable extends AbstractInvokable {
 
 Review comment:
   I think this class duplicates `NoOpInvokable`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369043196
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
 ##########
 @@ -684,34 +677,9 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception {
 	 */
 	@Test(timeout = 10000L)
 	public void testTaskSubmission() throws Exception {
-		final JobMasterId jobMasterId = JobMasterId.generate();
-		final AllocationID allocationId = new AllocationID();
-		final TaskDeploymentDescriptor taskDeploymentDescriptor = TaskDeploymentDescriptorBuilder
-			.newBuilder(jobId, TestInvokable.class)
-			.setAllocationId(allocationId)
-			.build();
-
-		final OneShotLatch taskInTerminalState = new OneShotLatch();
-		final TaskManagerActions taskManagerActions = createTaskManagerActionsWithTerminalStateTrigger(taskInTerminalState);
-		final JobManagerTable jobManagerTable = createJobManagerTableWithOneJob(jobMasterId, taskManagerActions);
-		final TaskExecutor taskExecutor = createTaskExecutorWithJobManagerTable(jobManagerTable);
-
-		try {
-			taskExecutor.start();
-
-			final TaskExecutorGateway taskExecutorGateway = taskExecutor.getSelfGateway(TaskExecutorGateway.class);
-			final JobMasterGateway jobMasterGateway = jobManagerTable.get(jobId).getJobManagerGateway();
-			requestSlotFromTaskExecutor(taskExecutorGateway, jobMasterGateway, allocationId);
-
-			taskExecutorGateway.submitTask(taskDeploymentDescriptor, jobMasterId, timeout);
-
-			CompletableFuture<Boolean> completionFuture = TestInvokable.COMPLETABLE_FUTURE;
-
-			completionFuture.get();
-
-			taskInTerminalState.await();
-		} finally {
-			RpcUtils.terminateRpcEndpoint(taskExecutor, timeout);
+		try (TaskExecutorTestingContext submissionContext = createTaskExecutorTestingContext(1)) {
+			submissionContext.startAllocateSlotAndSubmit(TestInvokable.class);
+			TestInvokable.COMPLETABLE_FUTURE.get();
 
 Review comment:
   How is this task different to `TaskExecutorSubmissionTest#testTaskSubmission`?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369016459
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private CompletableFuture<Void> closingFuture;
+
+	/** {@link Executor} to schedule internal calls to the main thread. */
+	private Executor mainThreadExecutor;
+
+	public TaskSlotTableImpl(
+		final int numberSlots,
+		final ResourceProfile totalAvailableResourceProfile,
+		final ResourceProfile defaultSlotResourceProfile,
+		final int memoryPageSize,
+		final TimerService<AllocationID> timerService) {
+
+		Preconditions.checkArgument(0 < numberSlots, "The number of task slots must be greater than 0.");
+
+		this.numberSlots = numberSlots;
+		this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+		this.memoryPageSize = memoryPageSize;
+
+		this.taskSlots = new HashMap<>(numberSlots);
+
+		this.timerService = Preconditions.checkNotNull(timerService);
+
+		budgetManager = new ResourceBudgetManager(Preconditions.checkNotNull(totalAvailableResourceProfile));
+
+		allocatedSlots = new HashMap<>(numberSlots);
+
+		taskSlotMappings = new HashMap<>(4 * numberSlots);
+
+		slotsPerJob = new HashMap<>(4);
+
+		slotActions = null;
+		state = State.CREATED;
+	}
+
+	@Override
+	public void start(SlotActions initialSlotActions, Executor mainThreadExecutor) {
+		Preconditions.checkState(
+			state == State.CREATED,
+			"The %s has to be just created before starting",
+			TaskSlotTableImpl.class.getSimpleName());
+		this.slotActions = Preconditions.checkNotNull(initialSlotActions);
+		this.mainThreadExecutor = Preconditions.checkNotNull(mainThreadExecutor);
+
+		timerService.start(this);
+
+		state = State.RUNNING;
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		if (state == State.CREATED) {
+			state = State.CLOSED;
+			closingFuture = CompletableFuture.completedFuture(null);
+		} else if (state == State.RUNNING) {
+			state = State.CLOSING;
+			final FlinkException cause = new FlinkException("Closing task slot table");
+			closingFuture = FutureUtils
+				.waitForAll(
+					new ArrayList<>(allocatedSlots.values())
+						.stream()
+						.map(slot -> freeSlot(slot, cause))
+						.collect(Collectors.toList()))
+				.thenRunAsync(
+					() -> {
+						state = State.CLOSED;
+						timerService.stop();
+					},
+					mainThreadExecutor);
+		}
+		return closingFuture;
+	}
+
+	@VisibleForTesting
+	public boolean isClosed() {
+		return state == State.CLOSED &&
+			taskSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown()) &&
+			allocatedSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown());
 
 Review comment:
   I don't understand these two conditions. Shouldn't the `TaskSlotTable` be closed if the state is `CLOSED`? Why checking the state of the memory managers of the slots here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   
   <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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368607984
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -356,7 +373,7 @@ public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundExce
 	 * @return True if the slot could be marked inactive
 	 */
 	public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) throws SlotNotFoundException {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   As discussed offline, enabling this check would need at least disabling message handling after `RpcEndpoint#onStop` to not accept disconnect messages from job master after `taskExecutor#onStop`. On the other hand, requiring only start method call (not `CREATED`) should be enough because calling it in non-running state will register a timeout in worse case or observe no slot. The issue with concurrent access in `timerService#stop` should be resolved anyways.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362826032
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTest.java
 ##########
 @@ -0,0 +1,52 @@
+package org.apache.flink.runtime.taskexecutor.slot;/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.junit.Test;
+
+import java.util.concurrent.CompletableFuture;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class TaskSlotTest {
 
 Review comment:
   `extends TestLogger` is missing.

----------------------------------------------------------------
This is an automated message from the 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r365833036
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -243,8 +252,181 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 		}
 	}
 
+	@Test
+	public void testAllocateSlot() {
+		final JobID jobId = new JobID();
+		final AllocationID allocationId = new AllocationID();
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithAllocatedSlot(
+				jobId,
+				allocationId,
+				new TestingSlotActionsBuilder().build());
+			Iterator<TaskSlot> allocatedSlots = taskSlotTable.getAllocatedSlots(jobId);
+			TaskSlot nextSlot = allocatedSlots.next();
+			assertThat(nextSlot.getIndex(), is(0));
+			assertThat(nextSlot.getAllocationId(), is(allocationId));
+			assertThat(nextSlot.getJobId(), is(jobId));
+			assertThat(allocatedSlots.hasNext(), is(false));
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test
+	public void testAddTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+			Iterator<Task> tasks = taskSlotTable.getTasks(jobId);
+			Task nextTask = tasks.next();
+			assertThat(nextTask.getExecutionId(), is(executionAttemptId));
+			assertThat(nextTask.getAllocationId(), is(allocationId));
+			assertThat(tasks.hasNext(), is(false));
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
 
 Review comment:
   I removed the unnecessary calls, also for subsequent comments and added comment in code for one where I think it was needed.

----------------------------------------------------------------
This is an automated message from the 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368609623
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -605,7 +615,7 @@ public boolean addTask(Task task) throws SlotNotFoundException, SlotNotActiveExc
 	 * @return The removed task if there is any for the given execution attempt id; otherwise null
 	 */
 	public Task removeTask(ExecutionAttemptID executionAttemptID) {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   All `TaskExecutor#updateTaskExecutionState` callbacks with `removeTask` should be scheduled to the main thread in task executor before task termination future is competed. This means that the final `CLOSED` table state will not be set before that. If so, I would keep the check.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369017114
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private CompletableFuture<Void> closingFuture;
+
+	/** {@link Executor} to schedule internal calls to the main thread. */
+	private Executor mainThreadExecutor;
+
+	public TaskSlotTableImpl(
+		final int numberSlots,
+		final ResourceProfile totalAvailableResourceProfile,
+		final ResourceProfile defaultSlotResourceProfile,
+		final int memoryPageSize,
+		final TimerService<AllocationID> timerService) {
+
+		Preconditions.checkArgument(0 < numberSlots, "The number of task slots must be greater than 0.");
+
+		this.numberSlots = numberSlots;
+		this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+		this.memoryPageSize = memoryPageSize;
+
+		this.taskSlots = new HashMap<>(numberSlots);
+
+		this.timerService = Preconditions.checkNotNull(timerService);
+
+		budgetManager = new ResourceBudgetManager(Preconditions.checkNotNull(totalAvailableResourceProfile));
+
+		allocatedSlots = new HashMap<>(numberSlots);
+
+		taskSlotMappings = new HashMap<>(4 * numberSlots);
+
+		slotsPerJob = new HashMap<>(4);
+
+		slotActions = null;
+		state = State.CREATED;
+	}
+
+	@Override
+	public void start(SlotActions initialSlotActions, Executor mainThreadExecutor) {
+		Preconditions.checkState(
+			state == State.CREATED,
+			"The %s has to be just created before starting",
+			TaskSlotTableImpl.class.getSimpleName());
+		this.slotActions = Preconditions.checkNotNull(initialSlotActions);
+		this.mainThreadExecutor = Preconditions.checkNotNull(mainThreadExecutor);
+
+		timerService.start(this);
+
+		state = State.RUNNING;
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		if (state == State.CREATED) {
+			state = State.CLOSED;
+			closingFuture = CompletableFuture.completedFuture(null);
+		} else if (state == State.RUNNING) {
+			state = State.CLOSING;
+			final FlinkException cause = new FlinkException("Closing task slot table");
+			closingFuture = FutureUtils
+				.waitForAll(
+					new ArrayList<>(allocatedSlots.values())
+						.stream()
+						.map(slot -> freeSlot(slot, cause))
+						.collect(Collectors.toList()))
+				.thenRunAsync(
+					() -> {
+						state = State.CLOSED;
+						timerService.stop();
+					},
+					mainThreadExecutor);
+		}
+		return closingFuture;
+	}
+
+	@VisibleForTesting
+	public boolean isClosed() {
+		return state == State.CLOSED &&
+			taskSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown()) &&
+			allocatedSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown());
+	}
+
+	@Override
+	public Set<AllocationID> getAllocationIdsPerJob(JobID jobId) {
+		final Set<AllocationID> allocationIds = slotsPerJob.get(jobId);
+
+		if (allocationIds == null) {
+			return Collections.emptySet();
+		} else {
+			return Collections.unmodifiableSet(allocationIds);
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot report methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	public SlotReport createSlotReport(ResourceID resourceId) {
+		List<SlotStatus> slotStatuses = new ArrayList<>();
+
+		for (int i = 0; i < numberSlots; i++) {
+			SlotID slotId = new SlotID(resourceId, i);
+			SlotStatus slotStatus;
+			if (taskSlots.containsKey(i)) {
+				TaskSlot taskSlot = taskSlots.get(i);
+
+				slotStatus = new SlotStatus(
+					slotId,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+			} else {
+				slotStatus = new SlotStatus(
+					slotId,
+					defaultSlotResourceProfile,
+					null,
+					null);
+			}
+
+			slotStatuses.add(slotStatus);
+		}
+
+		for (TaskSlot taskSlot : allocatedSlots.values()) {
+			if (taskSlot.getIndex() < 0) {
+				SlotID slotID = SlotID.generateDynamicSlotID(resourceId);
+				SlotStatus slotStatus = new SlotStatus(
+					slotID,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+				slotStatuses.add(slotStatus);
+			}
+		}
+
+		final SlotReport slotReport = new SlotReport(slotStatuses);
+
+		return slotReport;
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, Time slotTimeout) {
+		return allocateSlot(index, jobId, allocationId, defaultSlotResourceProfile, slotTimeout);
+	}
+
+	@Override
+	public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, Time slotTimeout) {
+		checkRunning();
+
+		Preconditions.checkArgument(index < numberSlots);
+
+		TaskSlot taskSlot = allocatedSlots.get(allocationId);
+		if (taskSlot != null) {
+			LOG.info("Allocation ID {} is already allocated in {}.", allocationId, taskSlot);
+			return false;
+		}
+
+		if (taskSlots.containsKey(index)) {
+			TaskSlot duplicatedTaskSlot = taskSlots.get(index);
+			LOG.info("Slot with index {} already exist, with resource profile {}, job id {} and allocation id {}.",
+				index,
+				duplicatedTaskSlot.getResourceProfile(),
+				duplicatedTaskSlot.getJobId(),
+				duplicatedTaskSlot.getAllocationId());
+			return duplicatedTaskSlot.getJobId().equals(jobId) &&
+				duplicatedTaskSlot.getAllocationId().equals(allocationId);
+		} else if (allocatedSlots.containsKey(allocationId)) {
+			return true;
+		}
+
+		resourceProfile = index >= 0 ? defaultSlotResourceProfile : resourceProfile;
+
+		if (!budgetManager.reserve(resourceProfile)) {
+			LOG.info("Cannot allocate the requested resources. Trying to allocate {}, "
+					+ "while the currently remaining available resources are {}, total is {}.",
+				resourceProfile,
+				budgetManager.getAvailableBudget(),
+				budgetManager.getTotalBudget());
+			return false;
+		}
+
+		taskSlot = new TaskSlot(index, resourceProfile, memoryPageSize, jobId, allocationId);
+		if (index >= 0) {
+			taskSlots.put(index, taskSlot);
+		}
+
+		// update the allocation id to task slot map
+		allocatedSlots.put(allocationId, taskSlot);
+
+		// register a timeout for this slot since it's in state allocated
+		timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+		// add this slot to the set of job slots
+		Set<AllocationID> slots = slotsPerJob.get(jobId);
+
+		if (slots == null) {
+			slots = new HashSet<>(4);
+			slotsPerJob.put(jobId, slots);
+		}
+
+		slots.add(allocationId);
+
+		return true;
+	}
+
+	@Override
+	public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundException {
+		checkRunning();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markActive()) {
+				// unregister a potential timeout
+				LOG.info("Activate slot {}.", allocationId);
+
+				timerService.unregisterTimeout(allocationId);
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markInactive()) {
+				// register a timeout to free the slot
+				timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public int freeSlot(AllocationID allocationId) throws SlotNotFoundException {
+		return freeSlot(allocationId, new Exception("The task slot of this task is being freed."));
+	}
+
+	@Override
+	public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			return freeSlot(taskSlot, cause).isDone() ? taskSlot.getIndex() : -1;
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	private CompletableFuture<Void> freeSlot(TaskSlot taskSlot, Throwable cause) {
 
 Review comment:
   ```suggestion
   	private CompletableFuture<Void> freeSlotInternal(TaskSlot taskSlot, Throwable cause) {
   ```

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370160518
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
 ##########
 @@ -76,6 +81,9 @@
 	/** Allocation id of this slot. */
 	private final AllocationID allocationId;
 
+	/** The closing future is completed when the slot is freed and closed. */
+	private CompletableFuture<Void> closingFuture;
 
 Review comment:
   nit: One could add `@Nullable` here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368071764
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
 ##########
 @@ -701,24 +701,22 @@ public void invoke() throws Exception {
 	}
 
 	@Test
-	public void testTaskInterruptionAndTerminationOnShutdown() throws Exception {
-		TaskExecutorTestingContext submissionContext = createTaskExecutorTestingContext(1);
+	public void testTaskSlotTableTerminationOnShutdown() throws Exception {
+		CompletableFuture<Void> taskSlotTableClosingFuture = new CompletableFuture<>();
+		TaskExecutorTestingContext submissionContext = createTaskExecutorTestingContext(
+			StubTaskSlotTable.newBuilder().closeAsyncReturns(taskSlotTableClosingFuture).build());
+		final CompletableFuture<Void> taskExecutorTerminationFuture;
 		try {
-			submissionContext.startAllocateSlotAndSubmit(TestInterruptableInvokable.class);
-			TestInterruptableInvokable.waitForStarted();
+			submissionContext.start();
 		} finally {
-			submissionContext.taskExecutor.closeAsync();
+			taskExecutorTerminationFuture = submissionContext.taskExecutor.closeAsync();
 		}
 
-		// check task has been interrupted
-		TestInterruptableInvokable.waitForInterrupted();
-
 		// check task executor is waiting for the task completion and has not terminated yet
-		final CompletableFuture<Void> taskExecutorTerminationFuture = submissionContext.taskExecutor.getTerminationFuture();
 		assertThat(taskExecutorTerminationFuture.isDone(), is(false));
 
-		// check task executor has exited after task completion
-		TestInterruptableInvokable.finish();
+		// check task executor has exited after task slot table termination
+		taskSlotTableClosingFuture.complete(null);
 
 Review comment:
   Nice work :-)

----------------------------------------------------------------
This is an automated message from the 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] xintongsong commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r361274776
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -403,54 +410,60 @@ public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFo
 		TaskSlot taskSlot = getTaskSlot(allocationId);
 
 		if (taskSlot != null) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Free slot {}.", taskSlot, cause);
-			} else {
-				LOG.info("Free slot {}.", taskSlot);
-			}
+			return freeSlot(taskSlot, cause);
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
 
-			final JobID jobId = taskSlot.getJobId();
+	private int freeSlot(TaskSlot taskSlot, Throwable cause) {
+		AllocationID allocationId = taskSlot.getAllocationId();
 
-			if (taskSlot.isEmpty()) {
-				// remove the allocation id to task slot mapping
-				allocatedSlots.remove(allocationId);
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Free slot {}.", taskSlot, cause);
+		} else {
+			LOG.info("Free slot {}.", taskSlot);
+		}
 
-				// unregister a potential timeout
-				timerService.unregisterTimeout(allocationId);
+		final JobID jobId = taskSlot.getJobId();
 
-				Set<AllocationID> slots = slotsPerJob.get(jobId);
+		if (taskSlot.isEmpty()) {
+			// remove the allocation id to task slot mapping
+			allocatedSlots.remove(allocationId);
 
-				if (slots == null) {
-					throw new IllegalStateException("There are no more slots allocated for the job " + jobId +
-						". This indicates a programming bug.");
-				}
+			// unregister a potential timeout
+			timerService.unregisterTimeout(allocationId);
 
-				slots.remove(allocationId);
+			Set<AllocationID> slots = slotsPerJob.get(jobId);
 
-				if (slots.isEmpty()) {
-					slotsPerJob.remove(jobId);
-				}
+			if (slots == null) {
+				throw new IllegalStateException("There are no more slots allocated for the job " + jobId +
+					". This indicates a programming bug.");
+			}
 
-				taskSlot.close();
-				taskSlots.remove(taskSlot.getIndex());
-				budgetManager.release(taskSlot.getResourceProfile());
+			slots.remove(allocationId);
 
-				return taskSlot.getIndex();
-			} else {
-				// we couldn't free the task slot because it still contains task, fail the tasks
-				// and set the slot state to releasing so that it gets eventually freed
-				taskSlot.markReleasing();
+			if (slots.isEmpty()) {
+				slotsPerJob.remove(jobId);
+			}
 
-				Iterator<Task> taskIterator = taskSlot.getTasks();
+			taskSlots.remove(taskSlot.getIndex());
+			budgetManager.release(taskSlot.getResourceProfile());
+			taskSlot.close();
 
-				while (taskIterator.hasNext()) {
-					taskIterator.next().failExternally(cause);
-				}
+			return taskSlot.getIndex();
+		} else {
+			// we couldn't free the task slot because it still contains task, fail the tasks
+			// and set the slot state to releasing so that it gets eventually freed
+			taskSlot.markReleasing();
+
+			Iterator<Task> taskIterator = taskSlot.getTasks();
 
-				return -1;
+			while (taskIterator.hasNext()) {
+				taskIterator.next().failExternally(cause);
 			}
-		} else {
-			throw new SlotNotFoundException(allocationId);
+
+			return -1;
 		}
 	}
 
 Review comment:
   Minor: This seems to be a irrelevant refactoring to me, which would be better in a separated hotfix commit.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362841392
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -243,8 +252,181 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 		}
 	}
 
+	@Test
+	public void testAllocateSlot() {
+		final JobID jobId = new JobID();
+		final AllocationID allocationId = new AllocationID();
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithAllocatedSlot(
+				jobId,
+				allocationId,
+				new TestingSlotActionsBuilder().build());
+			Iterator<TaskSlot> allocatedSlots = taskSlotTable.getAllocatedSlots(jobId);
+			TaskSlot nextSlot = allocatedSlots.next();
+			assertThat(nextSlot.getIndex(), is(0));
+			assertThat(nextSlot.getAllocationId(), is(allocationId));
+			assertThat(nextSlot.getJobId(), is(jobId));
+			assertThat(allocatedSlots.hasNext(), is(false));
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test
+	public void testAddTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+			Iterator<Task> tasks = taskSlotTable.getTasks(jobId);
+			Task nextTask = tasks.next();
+			assertThat(nextTask.getExecutionId(), is(executionAttemptId));
+			assertThat(nextTask.getAllocationId(), is(allocationId));
+			assertThat(tasks.hasNext(), is(false));
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test(timeout = 10000)
+	public void testRemoveTaskCallsFreeSlotAction() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		CompletableFuture<AllocationID> freeSlotFuture = new CompletableFuture<>();
+		SlotActions slotActions = new TestingSlotActions(freeSlotFuture::complete, (aid, uid) -> {});
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, slotActions);
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
+			assertThat(freeSlotFuture.get(), is(allocationId));
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test(timeout = 10000)
+	public void testFreeSlotInterruptsSubmittedTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			TestInterruptableInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+			TestInterruptableInvokable.STARTED_FUTURE.get();
+			assertThat(taskSlotTable.freeSlot(allocationId), is(-1));
+			TestInterruptableInvokable.INTERRUPTED_FUTURE.get();
+			CompletableFuture<Void> slotFreeFuture = taskSlotTable.freeAllSlots(null);
+			assertThat(slotFreeFuture.isDone(), is(false));
+			TestInterruptableInvokable.DONE_FUTURE.complete(null);
+			taskSlotTable.removeTask(executionAttemptId);
 
 Review comment:
   Why is this 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368607984
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -356,7 +373,7 @@ public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundExce
 	 * @return True if the slot could be marked inactive
 	 */
 	public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) throws SlotNotFoundException {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   As discussed offline, enabling this check would need at least disabling message handling after `RpcEndpoint#onStop` ([FLINK-15693](https://issues.apache.org/jira/browse/FLINK-15693)) to not accept disconnect messages from job master after `taskExecutor#onStop`. On the other hand, requiring only start method call (not `CREATED`) should be enough because calling it in non-running state will register a timeout in worse case or observe no slot. The issue with concurrent access in `timerService#stop` should be resolved anyways.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362852268
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##########
 @@ -376,7 +372,7 @@ private void handleStartTaskExecutorServicesException(Exception e) throws Except
 
 		return FutureUtils
 			.runAfterwards(
-				taskCompletionTracker.failIncompleteTasksAndGetTerminationFuture(),
+				taskSlotTable.freeAllSlots(new FlinkException("The task executor is shutting down.")),
 
 Review comment:
   The current way of ensuring that all tasks are being terminated before the returned future is completed is super hard to understand because the code path is too convoluted and too many components depend on each other to make it happen. It took me actually quite some time to figure this out and I'm somewhat familiar with the code. Instead of introducing additional APIs which need to be called in a very specific manner, I would suggest that we simply call `taskSlotTable.closeAsync()` which returns a future which is completed once all slots are closed. A slot is closed if all its task have terminated. The information about a task being closed does not need to be passed into the `TaskSlotTable` via the `freeSlot` but we could simple take a look at the `Task#getTerminationFuture` in order to control when a `TaskSlot` is completely released.
   
   A good design principle is context independence. If possible a component should work independent of its context meaning that it tries to decouple it self as good as possible from other components. In this case, for example, we require that `TaskExecutor` calls `TaskSlotTable#freeSlot` after we have called `TaskSlotTable#freeAllSlots` and before we can call `close`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   
   <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] AHeise edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
AHeise edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-576580269
 
 
   > I think the safest way to handle timer service concurrency issues is to schedule its stopping to the main thread of task executor. Then there is only a thread-confined access to its internal container for registered callbacks.
   
   Would be awesome if you could use that and simplify the threading model. Let me know if you need any pointer.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d 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] AHeise removed a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
AHeise removed a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-576580269
 
 
   > I think the safest way to handle timer service concurrency issues is to schedule its stopping to the main thread of task executor. Then there is only a thread-confined access to its internal container for registered callbacks.
   
   Would be awesome if you could use that and simplify the threading model. Let me know if you need any pointer.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370171931
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TestingTaskSlotTable.java
 ##########
 @@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import javax.annotation.Nullable;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Testing implementation of {@link TaskSlotTable} for tests.
+ */
+public class TestingTaskSlotTable<T extends TaskSlotPayload> implements TaskSlotTable<T> {
+	private final Supplier<SlotReport> createSlotReportSupplier;
+	private final Supplier<Boolean> allocateSlotSupplier;
+	private final BiFunction<JobID, AllocationID, Boolean> tryMarkSlotActiveBiFunction;
+	private final Function<T, Boolean> addTaskFunction;
+	private final Function<AllocationID, MemoryManager> memoryManagerGetter;
+	private final Supplier<CompletableFuture<Void>> closeAsyncSupplier;
+
+	private TestingTaskSlotTable(
+			Supplier<SlotReport> createSlotReportSupplier,
+			Supplier<Boolean> allocateSlotSupplier,
+			BiFunction<JobID, AllocationID, Boolean> tryMarkSlotActiveBiFunction,
+			Function<T, Boolean> addTaskFunction,
+			Function<AllocationID, MemoryManager> memoryManagerGetter,
+			Supplier<CompletableFuture<Void>> closeAsyncSupplier) {
+		this.createSlotReportSupplier = createSlotReportSupplier;
+		this.allocateSlotSupplier = allocateSlotSupplier;
+		this.tryMarkSlotActiveBiFunction = tryMarkSlotActiveBiFunction;
+		this.addTaskFunction = addTaskFunction;
+		this.memoryManagerGetter = memoryManagerGetter;
+		this.closeAsyncSupplier = closeAsyncSupplier;
+	}
+
+	@Override
+	public void start(SlotActions initialSlotActions, ComponentMainThreadExecutor mainThreadExecutor) {
+
+	}
+
+	@Override
+	public Set<AllocationID> getAllocationIdsPerJob(JobID jobId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public SlotReport createSlotReport(ResourceID resourceId) {
+		if (createSlotReportSupplier != null) {
+			return createSlotReportSupplier.get();
+		} else {
+			throw new UnsupportedOperationException();
+		}
 
 Review comment:
   Nit: Instead of this pattern, I would rather set a default `createSlotReportSupplier` in the builder. If possible the default supplier could return something meaningful or it could fail with a more expressive exception.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 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] tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369017444
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private CompletableFuture<Void> closingFuture;
+
+	/** {@link Executor} to schedule internal calls to the main thread. */
+	private Executor mainThreadExecutor;
+
+	public TaskSlotTableImpl(
+		final int numberSlots,
+		final ResourceProfile totalAvailableResourceProfile,
+		final ResourceProfile defaultSlotResourceProfile,
+		final int memoryPageSize,
+		final TimerService<AllocationID> timerService) {
+
+		Preconditions.checkArgument(0 < numberSlots, "The number of task slots must be greater than 0.");
+
+		this.numberSlots = numberSlots;
+		this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+		this.memoryPageSize = memoryPageSize;
+
+		this.taskSlots = new HashMap<>(numberSlots);
+
+		this.timerService = Preconditions.checkNotNull(timerService);
+
+		budgetManager = new ResourceBudgetManager(Preconditions.checkNotNull(totalAvailableResourceProfile));
+
+		allocatedSlots = new HashMap<>(numberSlots);
+
+		taskSlotMappings = new HashMap<>(4 * numberSlots);
+
+		slotsPerJob = new HashMap<>(4);
+
+		slotActions = null;
+		state = State.CREATED;
+	}
+
+	@Override
+	public void start(SlotActions initialSlotActions, Executor mainThreadExecutor) {
+		Preconditions.checkState(
+			state == State.CREATED,
+			"The %s has to be just created before starting",
+			TaskSlotTableImpl.class.getSimpleName());
+		this.slotActions = Preconditions.checkNotNull(initialSlotActions);
+		this.mainThreadExecutor = Preconditions.checkNotNull(mainThreadExecutor);
+
+		timerService.start(this);
+
+		state = State.RUNNING;
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		if (state == State.CREATED) {
+			state = State.CLOSED;
+			closingFuture = CompletableFuture.completedFuture(null);
+		} else if (state == State.RUNNING) {
+			state = State.CLOSING;
+			final FlinkException cause = new FlinkException("Closing task slot table");
+			closingFuture = FutureUtils
+				.waitForAll(
+					new ArrayList<>(allocatedSlots.values())
+						.stream()
+						.map(slot -> freeSlot(slot, cause))
+						.collect(Collectors.toList()))
+				.thenRunAsync(
+					() -> {
+						state = State.CLOSED;
+						timerService.stop();
+					},
+					mainThreadExecutor);
+		}
+		return closingFuture;
+	}
+
+	@VisibleForTesting
+	public boolean isClosed() {
+		return state == State.CLOSED &&
+			taskSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown()) &&
+			allocatedSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown());
+	}
+
+	@Override
+	public Set<AllocationID> getAllocationIdsPerJob(JobID jobId) {
+		final Set<AllocationID> allocationIds = slotsPerJob.get(jobId);
+
+		if (allocationIds == null) {
+			return Collections.emptySet();
+		} else {
+			return Collections.unmodifiableSet(allocationIds);
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot report methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	public SlotReport createSlotReport(ResourceID resourceId) {
+		List<SlotStatus> slotStatuses = new ArrayList<>();
+
+		for (int i = 0; i < numberSlots; i++) {
+			SlotID slotId = new SlotID(resourceId, i);
+			SlotStatus slotStatus;
+			if (taskSlots.containsKey(i)) {
+				TaskSlot taskSlot = taskSlots.get(i);
+
+				slotStatus = new SlotStatus(
+					slotId,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+			} else {
+				slotStatus = new SlotStatus(
+					slotId,
+					defaultSlotResourceProfile,
+					null,
+					null);
+			}
+
+			slotStatuses.add(slotStatus);
+		}
+
+		for (TaskSlot taskSlot : allocatedSlots.values()) {
+			if (taskSlot.getIndex() < 0) {
+				SlotID slotID = SlotID.generateDynamicSlotID(resourceId);
+				SlotStatus slotStatus = new SlotStatus(
+					slotID,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+				slotStatuses.add(slotStatus);
+			}
+		}
+
+		final SlotReport slotReport = new SlotReport(slotStatuses);
+
+		return slotReport;
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, Time slotTimeout) {
+		return allocateSlot(index, jobId, allocationId, defaultSlotResourceProfile, slotTimeout);
+	}
+
+	@Override
+	public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, Time slotTimeout) {
+		checkRunning();
+
+		Preconditions.checkArgument(index < numberSlots);
+
+		TaskSlot taskSlot = allocatedSlots.get(allocationId);
+		if (taskSlot != null) {
+			LOG.info("Allocation ID {} is already allocated in {}.", allocationId, taskSlot);
+			return false;
+		}
+
+		if (taskSlots.containsKey(index)) {
+			TaskSlot duplicatedTaskSlot = taskSlots.get(index);
+			LOG.info("Slot with index {} already exist, with resource profile {}, job id {} and allocation id {}.",
+				index,
+				duplicatedTaskSlot.getResourceProfile(),
+				duplicatedTaskSlot.getJobId(),
+				duplicatedTaskSlot.getAllocationId());
+			return duplicatedTaskSlot.getJobId().equals(jobId) &&
+				duplicatedTaskSlot.getAllocationId().equals(allocationId);
+		} else if (allocatedSlots.containsKey(allocationId)) {
+			return true;
+		}
+
+		resourceProfile = index >= 0 ? defaultSlotResourceProfile : resourceProfile;
+
+		if (!budgetManager.reserve(resourceProfile)) {
+			LOG.info("Cannot allocate the requested resources. Trying to allocate {}, "
+					+ "while the currently remaining available resources are {}, total is {}.",
+				resourceProfile,
+				budgetManager.getAvailableBudget(),
+				budgetManager.getTotalBudget());
+			return false;
+		}
+
+		taskSlot = new TaskSlot(index, resourceProfile, memoryPageSize, jobId, allocationId);
+		if (index >= 0) {
+			taskSlots.put(index, taskSlot);
+		}
+
+		// update the allocation id to task slot map
+		allocatedSlots.put(allocationId, taskSlot);
+
+		// register a timeout for this slot since it's in state allocated
+		timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+		// add this slot to the set of job slots
+		Set<AllocationID> slots = slotsPerJob.get(jobId);
+
+		if (slots == null) {
+			slots = new HashSet<>(4);
+			slotsPerJob.put(jobId, slots);
+		}
+
+		slots.add(allocationId);
+
+		return true;
+	}
+
+	@Override
+	public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundException {
+		checkRunning();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markActive()) {
+				// unregister a potential timeout
+				LOG.info("Activate slot {}.", allocationId);
+
+				timerService.unregisterTimeout(allocationId);
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markInactive()) {
+				// register a timeout to free the slot
+				timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public int freeSlot(AllocationID allocationId) throws SlotNotFoundException {
+		return freeSlot(allocationId, new Exception("The task slot of this task is being freed."));
+	}
+
+	@Override
+	public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			return freeSlot(taskSlot, cause).isDone() ? taskSlot.getIndex() : -1;
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	private CompletableFuture<Void> freeSlot(TaskSlot taskSlot, Throwable cause) {
+		AllocationID allocationId = taskSlot.getAllocationId();
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Free slot {}.", taskSlot, cause);
+		} else {
+			LOG.info("Free slot {}.", taskSlot);
+		}
+
+		final JobID jobId = taskSlot.getJobId();
 
 Review comment:
   move this statement into the if body.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 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] tillrohrmann commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-573141077
 
 
   @azagrebin is the PR ready for another round of review?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369008460
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -383,9 +123,7 @@ public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) thr
 	 * @throws SlotNotFoundException if there is not task slot for the given allocation id
 	 * @return Index of the freed slot if the slot could be freed; otherwise -1
 	 */
-	public int freeSlot(AllocationID allocationId) throws SlotNotFoundException {
-		return freeSlot(allocationId, new Exception("The task slot of this task is being freed."));
-	}
+	int freeSlot(AllocationID allocationId) throws SlotNotFoundException;
 
 Review comment:
   Can't have this method have a default implementation?

----------------------------------------------------------------
This is an automated message from the 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] xintongsong commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r361275054
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -403,54 +410,60 @@ public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFo
 		TaskSlot taskSlot = getTaskSlot(allocationId);
 
 		if (taskSlot != null) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Free slot {}.", taskSlot, cause);
-			} else {
-				LOG.info("Free slot {}.", taskSlot);
-			}
+			return freeSlot(taskSlot, cause);
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
 
-			final JobID jobId = taskSlot.getJobId();
+	private int freeSlot(TaskSlot taskSlot, Throwable cause) {
+		AllocationID allocationId = taskSlot.getAllocationId();
 
-			if (taskSlot.isEmpty()) {
-				// remove the allocation id to task slot mapping
-				allocatedSlots.remove(allocationId);
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Free slot {}.", taskSlot, cause);
+		} else {
+			LOG.info("Free slot {}.", taskSlot);
+		}
 
-				// unregister a potential timeout
-				timerService.unregisterTimeout(allocationId);
+		final JobID jobId = taskSlot.getJobId();
 
-				Set<AllocationID> slots = slotsPerJob.get(jobId);
+		if (taskSlot.isEmpty()) {
+			// remove the allocation id to task slot mapping
+			allocatedSlots.remove(allocationId);
 
-				if (slots == null) {
-					throw new IllegalStateException("There are no more slots allocated for the job " + jobId +
-						". This indicates a programming bug.");
-				}
+			// unregister a potential timeout
+			timerService.unregisterTimeout(allocationId);
 
-				slots.remove(allocationId);
+			Set<AllocationID> slots = slotsPerJob.get(jobId);
 
-				if (slots.isEmpty()) {
-					slotsPerJob.remove(jobId);
-				}
+			if (slots == null) {
+				throw new IllegalStateException("There are no more slots allocated for the job " + jobId +
+					". This indicates a programming bug.");
+			}
 
-				taskSlot.close();
-				taskSlots.remove(taskSlot.getIndex());
-				budgetManager.release(taskSlot.getResourceProfile());
+			slots.remove(allocationId);
 
-				return taskSlot.getIndex();
-			} else {
-				// we couldn't free the task slot because it still contains task, fail the tasks
-				// and set the slot state to releasing so that it gets eventually freed
-				taskSlot.markReleasing();
+			if (slots.isEmpty()) {
+				slotsPerJob.remove(jobId);
+			}
 
-				Iterator<Task> taskIterator = taskSlot.getTasks();
+			taskSlots.remove(taskSlot.getIndex());
+			budgetManager.release(taskSlot.getResourceProfile());
+			taskSlot.close();
 
 Review comment:
   Any special reason moving `taskSlot.close()` after `taskSlots.remove()` and `budgetManager.release`?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 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] tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370155937
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -838,12 +837,12 @@ public void remove() {
 	}
 
 	/**
-	 * Iterator over all {@link Task} for a given job.
+	 * Iterator over all {@link TaskSlotPayload} for a given job.
 	 */
-	private final class TaskIterator implements Iterator<Task> {
-		private final Iterator<TaskSlot> taskSlotIterator;
+	private final class TaskIterator implements Iterator<T> {
 
 Review comment:
   nit: Could be renamed into `TaskSlotPayloadIterator`

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368067378
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -605,7 +615,7 @@ public boolean addTask(Task task) throws SlotNotFoundException, SlotNotActiveExc
 	 * @return The removed task if there is any for the given execution attempt id; otherwise null
 	 */
 	public Task removeTask(ExecutionAttemptID executionAttemptID) {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   Are we sure that `removeTask` can only be called when the state is started or closing? Can't it happen that we call `removeTask` if there is a late `TaskExecutor#updateTaskExecutionState` call?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369013987
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
 
 Review comment:
   `@Nullable` annotation is missing or we need a default 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369087687
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##########
 @@ -158,6 +160,7 @@ public void testTaskSubmission() throws Exception {
 			tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get();
 
 			taskRunningFuture.get();
+			taskFinishedFuture.get();
 
 Review comment:
   I think it is a leftover from the previous implementation which was needed for the the test to work. I will remove it

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370156343
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##########
 @@ -501,12 +502,6 @@ public boolean isRunning() {
 		return executionState == ExecutionState.RUNNING;
 	}
 
-	/**
-	 * If the task has failed, this method gets the exception that caused this task to fail.
-	 * Otherwise this method returns null.
-	 *
-	 * @return The exception that caused the task to fail, or null, if the task has not failed.
-	 */
 
 Review comment:
   Why did you remove the JavaDocs?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 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] tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369023971
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##########
 @@ -158,6 +160,7 @@ public void testTaskSubmission() throws Exception {
 			tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get();
 
 			taskRunningFuture.get();
+			taskFinishedFuture.get();
 
 Review comment:
   I think we are overloading the test case here. The initial test case simply tests that we can submit a task if we have an assigned slot. What do you want to test here exactly?

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369022521
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private CompletableFuture<Void> closingFuture;
+
+	/** {@link Executor} to schedule internal calls to the main thread. */
+	private Executor mainThreadExecutor;
+
+	public TaskSlotTableImpl(
+		final int numberSlots,
+		final ResourceProfile totalAvailableResourceProfile,
+		final ResourceProfile defaultSlotResourceProfile,
+		final int memoryPageSize,
+		final TimerService<AllocationID> timerService) {
+
+		Preconditions.checkArgument(0 < numberSlots, "The number of task slots must be greater than 0.");
+
+		this.numberSlots = numberSlots;
+		this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+		this.memoryPageSize = memoryPageSize;
+
+		this.taskSlots = new HashMap<>(numberSlots);
+
+		this.timerService = Preconditions.checkNotNull(timerService);
+
+		budgetManager = new ResourceBudgetManager(Preconditions.checkNotNull(totalAvailableResourceProfile));
+
+		allocatedSlots = new HashMap<>(numberSlots);
+
+		taskSlotMappings = new HashMap<>(4 * numberSlots);
+
+		slotsPerJob = new HashMap<>(4);
+
+		slotActions = null;
+		state = State.CREATED;
+	}
+
+	@Override
+	public void start(SlotActions initialSlotActions, Executor mainThreadExecutor) {
+		Preconditions.checkState(
+			state == State.CREATED,
+			"The %s has to be just created before starting",
+			TaskSlotTableImpl.class.getSimpleName());
+		this.slotActions = Preconditions.checkNotNull(initialSlotActions);
+		this.mainThreadExecutor = Preconditions.checkNotNull(mainThreadExecutor);
+
+		timerService.start(this);
+
+		state = State.RUNNING;
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		if (state == State.CREATED) {
+			state = State.CLOSED;
+			closingFuture = CompletableFuture.completedFuture(null);
+		} else if (state == State.RUNNING) {
+			state = State.CLOSING;
+			final FlinkException cause = new FlinkException("Closing task slot table");
+			closingFuture = FutureUtils
+				.waitForAll(
+					new ArrayList<>(allocatedSlots.values())
+						.stream()
+						.map(slot -> freeSlot(slot, cause))
+						.collect(Collectors.toList()))
+				.thenRunAsync(
+					() -> {
+						state = State.CLOSED;
+						timerService.stop();
+					},
+					mainThreadExecutor);
+		}
+		return closingFuture;
+	}
+
+	@VisibleForTesting
+	public boolean isClosed() {
+		return state == State.CLOSED &&
+			taskSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown()) &&
+			allocatedSlots.values().stream().allMatch(slot -> slot.getMemoryManager().isShutdown());
+	}
+
+	@Override
+	public Set<AllocationID> getAllocationIdsPerJob(JobID jobId) {
+		final Set<AllocationID> allocationIds = slotsPerJob.get(jobId);
+
+		if (allocationIds == null) {
+			return Collections.emptySet();
+		} else {
+			return Collections.unmodifiableSet(allocationIds);
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot report methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	public SlotReport createSlotReport(ResourceID resourceId) {
+		List<SlotStatus> slotStatuses = new ArrayList<>();
+
+		for (int i = 0; i < numberSlots; i++) {
+			SlotID slotId = new SlotID(resourceId, i);
+			SlotStatus slotStatus;
+			if (taskSlots.containsKey(i)) {
+				TaskSlot taskSlot = taskSlots.get(i);
+
+				slotStatus = new SlotStatus(
+					slotId,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+			} else {
+				slotStatus = new SlotStatus(
+					slotId,
+					defaultSlotResourceProfile,
+					null,
+					null);
+			}
+
+			slotStatuses.add(slotStatus);
+		}
+
+		for (TaskSlot taskSlot : allocatedSlots.values()) {
+			if (taskSlot.getIndex() < 0) {
+				SlotID slotID = SlotID.generateDynamicSlotID(resourceId);
+				SlotStatus slotStatus = new SlotStatus(
+					slotID,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+				slotStatuses.add(slotStatus);
+			}
+		}
+
+		final SlotReport slotReport = new SlotReport(slotStatuses);
+
+		return slotReport;
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, Time slotTimeout) {
+		return allocateSlot(index, jobId, allocationId, defaultSlotResourceProfile, slotTimeout);
+	}
+
+	@Override
+	public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, Time slotTimeout) {
+		checkRunning();
+
+		Preconditions.checkArgument(index < numberSlots);
+
+		TaskSlot taskSlot = allocatedSlots.get(allocationId);
+		if (taskSlot != null) {
+			LOG.info("Allocation ID {} is already allocated in {}.", allocationId, taskSlot);
+			return false;
+		}
+
+		if (taskSlots.containsKey(index)) {
+			TaskSlot duplicatedTaskSlot = taskSlots.get(index);
+			LOG.info("Slot with index {} already exist, with resource profile {}, job id {} and allocation id {}.",
+				index,
+				duplicatedTaskSlot.getResourceProfile(),
+				duplicatedTaskSlot.getJobId(),
+				duplicatedTaskSlot.getAllocationId());
+			return duplicatedTaskSlot.getJobId().equals(jobId) &&
+				duplicatedTaskSlot.getAllocationId().equals(allocationId);
+		} else if (allocatedSlots.containsKey(allocationId)) {
+			return true;
+		}
+
+		resourceProfile = index >= 0 ? defaultSlotResourceProfile : resourceProfile;
+
+		if (!budgetManager.reserve(resourceProfile)) {
+			LOG.info("Cannot allocate the requested resources. Trying to allocate {}, "
+					+ "while the currently remaining available resources are {}, total is {}.",
+				resourceProfile,
+				budgetManager.getAvailableBudget(),
+				budgetManager.getTotalBudget());
+			return false;
+		}
+
+		taskSlot = new TaskSlot(index, resourceProfile, memoryPageSize, jobId, allocationId);
+		if (index >= 0) {
+			taskSlots.put(index, taskSlot);
+		}
+
+		// update the allocation id to task slot map
+		allocatedSlots.put(allocationId, taskSlot);
+
+		// register a timeout for this slot since it's in state allocated
+		timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+		// add this slot to the set of job slots
+		Set<AllocationID> slots = slotsPerJob.get(jobId);
+
+		if (slots == null) {
+			slots = new HashSet<>(4);
+			slotsPerJob.put(jobId, slots);
+		}
+
+		slots.add(allocationId);
+
+		return true;
+	}
+
+	@Override
+	public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundException {
+		checkRunning();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markActive()) {
+				// unregister a potential timeout
+				LOG.info("Activate slot {}.", allocationId);
+
+				timerService.unregisterTimeout(allocationId);
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markInactive()) {
+				// register a timeout to free the slot
+				timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public int freeSlot(AllocationID allocationId) throws SlotNotFoundException {
+		return freeSlot(allocationId, new Exception("The task slot of this task is being freed."));
+	}
+
+	@Override
+	public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			return freeSlot(taskSlot, cause).isDone() ? taskSlot.getIndex() : -1;
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	private CompletableFuture<Void> freeSlot(TaskSlot taskSlot, Throwable cause) {
+		AllocationID allocationId = taskSlot.getAllocationId();
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Free slot {}.", taskSlot, cause);
+		} else {
+			LOG.info("Free slot {}.", taskSlot);
+		}
+
+		final JobID jobId = taskSlot.getJobId();
+
+		if (taskSlot.isEmpty()) {
+			// remove the allocation id to task slot mapping
+			allocatedSlots.remove(allocationId);
+
+			// unregister a potential timeout
+			timerService.unregisterTimeout(allocationId);
+
+			Set<AllocationID> slots = slotsPerJob.get(jobId);
+
+			if (slots == null) {
+				throw new IllegalStateException("There are no more slots allocated for the job " + jobId +
+					". This indicates a programming bug.");
+			}
+
+			slots.remove(allocationId);
+
+			if (slots.isEmpty()) {
+				slotsPerJob.remove(jobId);
+			}
+
+			taskSlots.remove(taskSlot.getIndex());
+			budgetManager.release(taskSlot.getResourceProfile());
+		}
+		return taskSlot.closeAsync(cause);
+	}
+
+	@Override
+	public boolean isValidTimeout(AllocationID allocationId, UUID ticket) {
+		checkStarted();
+
+		return state == State.RUNNING && timerService.isValid(allocationId, ticket);
+	}
+
+	@Override
+	public boolean isAllocated(int index, JobID jobId, AllocationID allocationId) {
+		TaskSlot taskSlot = taskSlots.get(index);
+		if (taskSlot != null) {
+			return taskSlot.isAllocated(jobId, allocationId);
+		} else if (index < 0) {
+			return allocatedSlots.containsKey(allocationId);
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean tryMarkSlotActive(JobID jobId, AllocationID allocationId) {
+		TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null && taskSlot.isAllocated(jobId, allocationId)) {
+			return taskSlot.markActive();
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean isSlotFree(int index) {
+		return !taskSlots.containsKey(index);
+	}
+
+	@Override
+	public boolean hasAllocatedSlots(JobID jobId) {
+		return getAllocatedSlots(jobId).hasNext();
+	}
+
+	@Override
+	public Iterator<TaskSlot> getAllocatedSlots(JobID jobId) {
+		return new TaskSlotIterator(jobId, TaskSlotState.ALLOCATED);
+	}
+
+	@Override
+	public Iterator<AllocationID> getActiveSlots(JobID jobId) {
+		return new AllocationIDIterator(jobId, TaskSlotState.ACTIVE);
+	}
+
+	@Override
+	@Nullable
+	public JobID getOwningJob(AllocationID allocationId) {
+		final TaskSlot taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			return taskSlot.getJobId();
+		} else {
+			return null;
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// Task methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	public boolean addTask(Task task) throws SlotNotFoundException, SlotNotActiveException {
+		checkRunning();
+		Preconditions.checkNotNull(task);
+
+		TaskSlot taskSlot = getTaskSlot(task.getAllocationId());
+
+		if (taskSlot != null) {
+			if (taskSlot.isActive(task.getJobID(), task.getAllocationId())) {
+				if (taskSlot.add(task)) {
+					taskSlotMappings.put(task.getExecutionId(), new TaskSlotMapping(task, taskSlot));
+
+					return true;
+				} else {
+					return false;
+				}
+			} else {
+				throw new SlotNotActiveException(task.getJobID(), task.getAllocationId());
+			}
+		} else {
+			throw new SlotNotFoundException(task.getAllocationId());
+		}
+	}
+
+	@Override
+	public Task removeTask(ExecutionAttemptID executionAttemptID) {
+		checkRunningOrClosing();
 
 Review comment:
   Should we replace this check with `checkStarted()`? That way we would not impose that all removal task calls come before finishing the close operation. Otherwise we would need to add a test which guards this assumption of the `Task`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362821873
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -146,12 +149,7 @@ public void start(SlotActions initialSlotActions) {
 	public void stop() {
 		started = false;
 		timerService.stop();
-		allocatedSlots
-			.values()
-			.stream()
-			.filter(slot -> !taskSlots.containsKey(slot.getIndex()))
-			.forEach(TaskSlot::close);
-		allocatedSlots.clear();
+		Preconditions.checkState(allocatedSlots.isEmpty(), "All slots have to be freed before stopping TaskSlotTable");
 
 Review comment:
   This precondition check could be moved to the beginning of this method.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362826624
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
 ##########
 @@ -77,6 +78,9 @@
 	/** Allocation id of this slot; null if not allocated. */
 
 Review comment:
   I guess `allocationId` is never `null`. Could update the comment in a separate commit.

----------------------------------------------------------------
This is an automated message from the 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368614904
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -686,7 +696,7 @@ public MemoryManager getTaskMemoryManager(AllocationID allocationID) throws Slot
 
 	@Override
 	public void notifyTimeout(AllocationID key, UUID ticket) {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   See answer in the comment to `isValidTimeout`

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370154908
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotPayload.java
 ##########
 @@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Task interface for {@link org.apache.flink.runtime.taskexecutor.slot.TaskSlot}.
+ */
+public interface TaskSlotPayload {
+	JobID getJobID();
+
+	ExecutionAttemptID getExecutionId();
+
+	AllocationID getAllocationId();
+
+	CompletableFuture<?> getTerminationFuture();
+
+	/**
+	 * Fail the owning task with the given throwable.
 
 Review comment:
   ```suggestion
   	 * Fail the payload with the given throwable. This operation should eventually complete the termination future.
   ```

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369046430
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
 ##########
 @@ -2044,6 +1907,97 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa
 			TaskManagerRunner.createBackPressureSampleService(configuration, rpc.getScheduledExecutor()));
 	}
 
+	private TaskExecutorTestingContext createTaskExecutorTestingContext(int numberOfSlots) {
+		return createTaskExecutorTestingContext(TaskSlotUtils.createTaskSlotTable(numberOfSlots));
+	}
+
+	private TaskExecutorTestingContext createTaskExecutorTestingContext(final TaskSlotTable taskSlotTable) {
+		final OneShotLatch offerSlotsLatch = new OneShotLatch();
+		final TestingJobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder()
+			.setOfferSlotsFunction((resourceID, slotOffers) -> {
+				offerSlotsLatch.trigger();
+				return CompletableFuture.completedFuture(slotOffers);
+			}).build();
+		final JobLeaderService jobLeaderService = new JobLeaderService(
+			taskManagerLocation,
+			RetryingRegistrationConfiguration.defaultConfiguration());
+		TaskExecutorLocalStateStoresManager stateStoresManager = null;
+		try {
+			stateStoresManager = createTaskExecutorLocalStateStoresManager();
+		} catch (IOException e) {
+			ExceptionUtils.rethrow(e);
+		}
+		final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(new TaskManagerServicesBuilder()
+			.setTaskSlotTable(taskSlotTable)
+			.setJobLeaderService(jobLeaderService)
+			.setTaskStateManager(stateStoresManager)
+			.build());
+		jobManagerLeaderRetriever.notifyListener(jobMasterGateway.getAddress(), jobMasterGateway.getFencingToken().toUUID());
+		rpc.registerGateway(jobMasterGateway.getAddress(), jobMasterGateway);
 
 Review comment:
   I would change the calls. Once we call `notifyListener` the endpoint should already be registered at the `RpcService`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362840658
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java
 ##########
 @@ -243,8 +252,181 @@ public void testGenerateSlotReport() throws SlotNotFoundException {
 		}
 	}
 
+	@Test
+	public void testAllocateSlot() {
+		final JobID jobId = new JobID();
+		final AllocationID allocationId = new AllocationID();
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithAllocatedSlot(
+				jobId,
+				allocationId,
+				new TestingSlotActionsBuilder().build());
+			Iterator<TaskSlot> allocatedSlots = taskSlotTable.getAllocatedSlots(jobId);
+			TaskSlot nextSlot = allocatedSlots.next();
+			assertThat(nextSlot.getIndex(), is(0));
+			assertThat(nextSlot.getAllocationId(), is(allocationId));
+			assertThat(nextSlot.getJobId(), is(jobId));
+			assertThat(allocatedSlots.hasNext(), is(false));
+		} finally {
+			stopTable(taskSlotTable);
+		}
+	}
+
+	@Test
+	public void testAddTask() throws Exception {
+		final JobID jobId = new JobID();
+		final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID();
+		final AllocationID allocationId = new AllocationID();
+		Task task = createTask(
+			jobId,
+			executionAttemptId,
+			allocationId,
+			EmptyInvokable.class);
+		TaskSlotTable taskSlotTable = null;
+		try {
+			taskSlotTable = createTaskSlotTableWithStartedTask(task, new TestingSlotActionsBuilder().build());
+			Iterator<Task> tasks = taskSlotTable.getTasks(jobId);
+			Task nextTask = tasks.next();
+			assertThat(nextTask.getExecutionId(), is(executionAttemptId));
+			assertThat(nextTask.getAllocationId(), is(allocationId));
+			assertThat(tasks.hasNext(), is(false));
+			taskSlotTable.freeSlot(allocationId);
+			taskSlotTable.removeTask(executionAttemptId);
 
 Review comment:
   why are these calls 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.
 
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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368064219
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -356,7 +373,7 @@ public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundExce
 	 * @return True if the slot could be marked inactive
 	 */
 	public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) throws SlotNotFoundException {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   Why do we allow the closing state here? I think we cannot do this because this might introduce a potential race condition between the `timerService` being stopped and registering a new timeout on 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   
   <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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r365814215
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
 ##########
 @@ -731,70 +697,6 @@ public void invoke() throws Exception {
 		}
 	}
 
-	@Test
-	public void testTaskInterruptionAndTerminationOnShutdown() throws Exception {
 
 Review comment:
   I returned the test but refactored it to use `StubTaskSlotTable` and checking waiting for `TaskSlotTable.closeAsync` while terminating `TaskExecutor`. Interruption of tasks in `TaskSlotTable.closeAsync` is tested separately.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r369013726
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,777 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl implements TaskSlotTable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private CompletableFuture<Void> closingFuture;
+
+	/** {@link Executor} to schedule internal calls to the main thread. */
+	private Executor mainThreadExecutor;
 
 Review comment:
   I would suggest to use a `ComponentMainThreadExecutor` instead of an `Executor` here. Moreover, this field should be initialized with something or marked as `@Nullable`. I would suggest to use
   
   ```
   new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor(
   			"TaskSlotTableImpl is not initialized with proper main thread executor. " +
   				"Call to Scheduler.start(...) required.");
   ```

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368059634
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
 ##########
 @@ -180,12 +180,6 @@ public void shutDown() throws FlinkException {
 			exception = ExceptionUtils.firstOrSuppressed(e, exception);
 		}
 
-		try {
-			taskSlotTable.stop();
-		} catch (Exception e) {
-			exception = ExceptionUtils.firstOrSuppressed(e, exception);
-		}
 
 Review comment:
   Not sure whether we should remove this as long as `taskSlotTable` stays part of `TaskManagerServices`. I think it would be better to call `taskSlotTable.close()` if we don't factor `taskSlotTable` out of `TaskManagerServices`.

----------------------------------------------------------------
This is an automated message from the 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] azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368614621
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -462,7 +471,7 @@ public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFo
 	 * @return True if the timeout is valid; otherwise false
 	 */
 	public boolean isValidTimeout(AllocationID allocationId, UUID ticket) {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   I think this would not hurt if we relax this check to `STARTED` (not `CREATED`) and make `RUNNING` state check part of the return value. This method is called after `notifyTimeout` where we can also relax it to `STARTED`. Even if we schedule `timerService#stop` to the main thread, `notifyTimeout` can be called afterwards because `timerService#stop` does not block on internal executor shutdown and we do not want it, I guess. If those methods are called afterwards, it should not be problem as they will just find that there is no registered callback anymore.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362835993
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
 ##########
 @@ -274,8 +282,10 @@ public String toString() {
 
 	@Override
 	public void close() {
+		Preconditions.checkState(tasks.isEmpty(), "All tasks have to be removed before closing the slot");
 
 Review comment:
   I think the current API design is cumbersome. Before closing a `TaskSlot`, we now always have to make sure that all `Tasks` have completely terminated and are removed from the `TaskSlot`. Why can't we say that we fail all remaining tasks, remove them from the slot once they are terminated and complete the `closingFuture` once this has happened?
   
   Please also take a look at `AutoCloseableAsync`. I think it makes sense to let this class implement this interface.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d 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] tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370154578
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotPayload.java
 ##########
 @@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Task interface for {@link org.apache.flink.runtime.taskexecutor.slot.TaskSlot}.
 
 Review comment:
   ```suggestion
    * Payload interface for {@link org.apache.flink.runtime.taskexecutor.slot.TaskSlot}.
   ```

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368070093
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/TestInterruptableInvokable.java
 ##########
 @@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testtasks;
+
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test invokable which completes the given future when interrupted (can be used only once).
+ */
+public class TestInterruptableInvokable extends AbstractInvokable {
+	private static CompletableFuture<Void> interruptedFuture;
+	private static CompletableFuture<Void> startedFuture;
+	private static CompletableFuture<Void> doneFuture;
 
 Review comment:
   Could be `OneShotLatches` instead of `CompletableFutures`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145789064 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145789064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370163550
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##########
 @@ -819,4 +820,21 @@ static TaskDeploymentDescriptor createTaskDeploymentDescriptor(
 			producedPartitions,
 			inputGates);
 	}
+
+	/**
+	 * Test invokable which completes the given future when executed.
+	 */
+	public static class TestInvokable extends AbstractInvokable {
 
 Review comment:
   Does it need to be public? If yes, then I would suggest to give it a more meaningful name. Maybe `FutureCompletingInvokable`.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r370162722
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImpl.java
 ##########
 @@ -0,0 +1,779 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.slot;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceBudgetManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor.DummyComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+/**
+ * Default implementation of {@link TaskSlotTable}.
+ */
+public class TaskSlotTableImpl<T extends TaskSlotPayload> implements TaskSlotTable<T> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskSlotTableImpl.class);
+
+	/**
+	 * Number of slots in static slot allocation.
+	 * If slot is requested with an index, the requested index must within the range of [0, numberSlots).
+	 * When generating slot report, we should always generate slots with index in [0, numberSlots) even the slot does not exist.
+	 */
+	private final int numberSlots;
+
+	/** Slot resource profile for static slot allocation. */
+	private final ResourceProfile defaultSlotResourceProfile;
+
+	/** Page size for memory manager. */
+	private final int memoryPageSize;
+
+	/** Timer service used to time out allocated slots. */
+	private final TimerService<AllocationID> timerService;
+
+	/** The list of all task slots. */
+	private final Map<Integer, TaskSlot<T>> taskSlots;
+
+	/** Mapping from allocation id to task slot. */
+	private final Map<AllocationID, TaskSlot<T>> allocatedSlots;
+
+	/** Mapping from execution attempt id to task and task slot. */
+	private final Map<ExecutionAttemptID, TaskSlotMapping<T>> taskSlotMappings;
+
+	/** Mapping from job id to allocated slots for a job. */
+	private final Map<JobID, Set<AllocationID>> slotsPerJob;
+
+	/** Interface for slot actions, such as freeing them or timing them out. */
+	@Nullable
+	private SlotActions slotActions;
+
+	/** The table state. */
+	private volatile State state;
+
+	private final ResourceBudgetManager budgetManager;
+
+	/** The closing future is completed when all slot are freed and state is closed. */
+	private final CompletableFuture<Void> closingFuture;
+
+	/** {@link ComponentMainThreadExecutor} to schedule internal calls to the main thread. */
+	private ComponentMainThreadExecutor mainThreadExecutor = new DummyComponentMainThreadExecutor(
+		"TaskSlotTableImpl is not initialized with proper main thread executor, " +
+			"call to TaskSlotTableImpl#start is required");
+
+	public TaskSlotTableImpl(
+			final int numberSlots,
+			final ResourceProfile totalAvailableResourceProfile,
+			final ResourceProfile defaultSlotResourceProfile,
+			final int memoryPageSize,
+			final TimerService<AllocationID> timerService) {
+		Preconditions.checkArgument(0 < numberSlots, "The number of task slots must be greater than 0.");
+
+		this.numberSlots = numberSlots;
+		this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+		this.memoryPageSize = memoryPageSize;
+
+		this.taskSlots = new HashMap<>(numberSlots);
+
+		this.timerService = Preconditions.checkNotNull(timerService);
+
+		budgetManager = new ResourceBudgetManager(Preconditions.checkNotNull(totalAvailableResourceProfile));
+
+		allocatedSlots = new HashMap<>(numberSlots);
+
+		taskSlotMappings = new HashMap<>(4 * numberSlots);
+
+		slotsPerJob = new HashMap<>(4);
+
+		slotActions = null;
+		state = State.CREATED;
+		closingFuture = new CompletableFuture<>();
+	}
+
+	@Override
+	public void start(SlotActions initialSlotActions, ComponentMainThreadExecutor mainThreadExecutor) {
+		Preconditions.checkState(
+			state == State.CREATED,
+			"The %s has to be just created before starting",
+			TaskSlotTableImpl.class.getSimpleName());
+		this.slotActions = Preconditions.checkNotNull(initialSlotActions);
+		this.mainThreadExecutor = Preconditions.checkNotNull(mainThreadExecutor);
+
+		timerService.start(this);
+
+		state = State.RUNNING;
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		if (state == State.CREATED) {
+			state = State.CLOSED;
+			closingFuture.complete(null);
+		} else if (state == State.RUNNING) {
+			state = State.CLOSING;
+			final FlinkException cause = new FlinkException("Closing task slot table");
+			CompletableFuture<Void> cleanupFuture = FutureUtils
+				.waitForAll(
+					new ArrayList<>(allocatedSlots.values())
+						.stream()
+						.map(slot -> freeSlotInternal(slot, cause))
+						.collect(Collectors.toList()))
+				.thenRunAsync(
+					() -> {
+						state = State.CLOSED;
+						timerService.stop();
+					},
+					mainThreadExecutor);
+			FutureUtils.forward(cleanupFuture, closingFuture);
+		}
+		return closingFuture;
+	}
+
+	@VisibleForTesting
+	public boolean isClosed() {
+		return state == State.CLOSED;
+	}
+
+	@Override
+	public Set<AllocationID> getAllocationIdsPerJob(JobID jobId) {
+		final Set<AllocationID> allocationIds = slotsPerJob.get(jobId);
+
+		if (allocationIds == null) {
+			return Collections.emptySet();
+		} else {
+			return Collections.unmodifiableSet(allocationIds);
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot report methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	public SlotReport createSlotReport(ResourceID resourceId) {
+		List<SlotStatus> slotStatuses = new ArrayList<>();
+
+		for (int i = 0; i < numberSlots; i++) {
+			SlotID slotId = new SlotID(resourceId, i);
+			SlotStatus slotStatus;
+			if (taskSlots.containsKey(i)) {
+				TaskSlot<T> taskSlot = taskSlots.get(i);
+
+				slotStatus = new SlotStatus(
+					slotId,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+			} else {
+				slotStatus = new SlotStatus(
+					slotId,
+					defaultSlotResourceProfile,
+					null,
+					null);
+			}
+
+			slotStatuses.add(slotStatus);
+		}
+
+		for (TaskSlot<T> taskSlot : allocatedSlots.values()) {
+			if (taskSlot.getIndex() < 0) {
+				SlotID slotID = SlotID.generateDynamicSlotID(resourceId);
+				SlotStatus slotStatus = new SlotStatus(
+					slotID,
+					taskSlot.getResourceProfile(),
+					taskSlot.getJobId(),
+					taskSlot.getAllocationId());
+				slotStatuses.add(slotStatus);
+			}
+		}
+
+		final SlotReport slotReport = new SlotReport(slotStatuses);
+
+		return slotReport;
+	}
+
+	// ---------------------------------------------------------------------
+	// Slot methods
+	// ---------------------------------------------------------------------
+
+	@VisibleForTesting
+	@Override
+	public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, Time slotTimeout) {
+		return allocateSlot(index, jobId, allocationId, defaultSlotResourceProfile, slotTimeout);
+	}
+
+	@Override
+	public boolean allocateSlot(
+			int index,
+			JobID jobId,
+			AllocationID allocationId,
+			ResourceProfile resourceProfile,
+			Time slotTimeout) {
+		checkRunning();
+
+		Preconditions.checkArgument(index < numberSlots);
+
+		TaskSlot<T> taskSlot = allocatedSlots.get(allocationId);
+		if (taskSlot != null) {
+			LOG.info("Allocation ID {} is already allocated in {}.", allocationId, taskSlot);
+			return false;
+		}
+
+		if (taskSlots.containsKey(index)) {
+			TaskSlot<T> duplicatedTaskSlot = taskSlots.get(index);
+			LOG.info("Slot with index {} already exist, with resource profile {}, job id {} and allocation id {}.",
+				index,
+				duplicatedTaskSlot.getResourceProfile(),
+				duplicatedTaskSlot.getJobId(),
+				duplicatedTaskSlot.getAllocationId());
+			return duplicatedTaskSlot.getJobId().equals(jobId) &&
+				duplicatedTaskSlot.getAllocationId().equals(allocationId);
+		} else if (allocatedSlots.containsKey(allocationId)) {
+			return true;
+		}
+
+		resourceProfile = index >= 0 ? defaultSlotResourceProfile : resourceProfile;
+
+		if (!budgetManager.reserve(resourceProfile)) {
+			LOG.info("Cannot allocate the requested resources. Trying to allocate {}, "
+					+ "while the currently remaining available resources are {}, total is {}.",
+				resourceProfile,
+				budgetManager.getAvailableBudget(),
+				budgetManager.getTotalBudget());
+			return false;
+		}
+
+		taskSlot = new TaskSlot<>(index, resourceProfile, memoryPageSize, jobId, allocationId);
+		if (index >= 0) {
+			taskSlots.put(index, taskSlot);
+		}
+
+		// update the allocation id to task slot map
+		allocatedSlots.put(allocationId, taskSlot);
+
+		// register a timeout for this slot since it's in state allocated
+		timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+		// add this slot to the set of job slots
+		Set<AllocationID> slots = slotsPerJob.get(jobId);
+
+		if (slots == null) {
+			slots = new HashSet<>(4);
+			slotsPerJob.put(jobId, slots);
+		}
+
+		slots.add(allocationId);
+
+		return true;
+	}
+
+	@Override
+	public boolean markSlotActive(AllocationID allocationId) throws SlotNotFoundException {
+		checkRunning();
+
+		TaskSlot<T> taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markActive()) {
+				// unregister a potential timeout
+				LOG.info("Activate slot {}.", allocationId);
+
+				timerService.unregisterTimeout(allocationId);
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public boolean markSlotInactive(AllocationID allocationId, Time slotTimeout) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot<T> taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			if (taskSlot.markInactive()) {
+				// register a timeout to free the slot
+				timerService.registerTimeout(allocationId, slotTimeout.getSize(), slotTimeout.getUnit());
+
+				return true;
+			} else {
+				return false;
+			}
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	@Override
+	public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFoundException {
+		checkStarted();
+
+		TaskSlot<T> taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			return freeSlotInternal(taskSlot, cause).isDone() ? taskSlot.getIndex() : -1;
+		} else {
+			throw new SlotNotFoundException(allocationId);
+		}
+	}
+
+	private CompletableFuture<Void> freeSlotInternal(TaskSlot<T> taskSlot, Throwable cause) {
+		AllocationID allocationId = taskSlot.getAllocationId();
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Free slot {}.", taskSlot, cause);
+		} else {
+			LOG.info("Free slot {}.", taskSlot);
+		}
+
+		if (taskSlot.isEmpty()) {
+			// remove the allocation id to task slot mapping
+			allocatedSlots.remove(allocationId);
+
+			// unregister a potential timeout
+			timerService.unregisterTimeout(allocationId);
+
+			JobID jobId = taskSlot.getJobId();
+			Set<AllocationID> slots = slotsPerJob.get(jobId);
+
+			if (slots == null) {
+				throw new IllegalStateException("There are no more slots allocated for the job " + jobId +
+					". This indicates a programming bug.");
+			}
+
+			slots.remove(allocationId);
+
+			if (slots.isEmpty()) {
+				slotsPerJob.remove(jobId);
+			}
+
+			taskSlots.remove(taskSlot.getIndex());
+			budgetManager.release(taskSlot.getResourceProfile());
+		}
+		return taskSlot.closeAsync(cause);
+	}
+
+	@Override
+	public boolean isValidTimeout(AllocationID allocationId, UUID ticket) {
+		checkStarted();
+
+		return state == State.RUNNING && timerService.isValid(allocationId, ticket);
+	}
+
+	@Override
+	public boolean isAllocated(int index, JobID jobId, AllocationID allocationId) {
+		TaskSlot<T> taskSlot = taskSlots.get(index);
+		if (taskSlot != null) {
+			return taskSlot.isAllocated(jobId, allocationId);
+		} else if (index < 0) {
+			return allocatedSlots.containsKey(allocationId);
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean tryMarkSlotActive(JobID jobId, AllocationID allocationId) {
+		TaskSlot<T> taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null && taskSlot.isAllocated(jobId, allocationId)) {
+			return taskSlot.markActive();
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean isSlotFree(int index) {
+		return !taskSlots.containsKey(index);
+	}
+
+	@Override
+	public boolean hasAllocatedSlots(JobID jobId) {
+		return getAllocatedSlots(jobId).hasNext();
+	}
+
+	@Override
+	public Iterator<TaskSlot<T>> getAllocatedSlots(JobID jobId) {
+		return new TaskSlotIterator(jobId, TaskSlotState.ALLOCATED);
+	}
+
+	@Override
+	public Iterator<AllocationID> getActiveSlots(JobID jobId) {
+		return new AllocationIDIterator(jobId, TaskSlotState.ACTIVE);
+	}
+
+	@Override
+	@Nullable
+	public JobID getOwningJob(AllocationID allocationId) {
+		final TaskSlot<T> taskSlot = getTaskSlot(allocationId);
+
+		if (taskSlot != null) {
+			return taskSlot.getJobId();
+		} else {
+			return null;
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// Task methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	public boolean addTask(T task) throws SlotNotFoundException, SlotNotActiveException {
+		checkRunning();
+		Preconditions.checkNotNull(task);
+
+		TaskSlot<T> taskSlot = getTaskSlot(task.getAllocationId());
+
+		if (taskSlot != null) {
+			if (taskSlot.isActive(task.getJobID(), task.getAllocationId())) {
+				if (taskSlot.add(task)) {
+					taskSlotMappings.put(task.getExecutionId(), new TaskSlotMapping<>(task, taskSlot));
+
+					return true;
+				} else {
+					return false;
+				}
+			} else {
+				throw new SlotNotActiveException(task.getJobID(), task.getAllocationId());
+			}
+		} else {
+			throw new SlotNotFoundException(task.getAllocationId());
+		}
+	}
+
+	@Override
+	public T removeTask(ExecutionAttemptID executionAttemptID) {
+		checkStarted();
+
+		TaskSlotMapping<T> taskSlotMapping = taskSlotMappings.remove(executionAttemptID);
+
+		if (taskSlotMapping != null) {
+			T task = taskSlotMapping.getTask();
+			TaskSlot<T> taskSlot = taskSlotMapping.getTaskSlot();
+
+			taskSlot.remove(task.getExecutionId());
+
+			if (taskSlot.isReleasing() && taskSlot.isEmpty()) {
+				slotActions.freeSlot(taskSlot.getAllocationId());
+			}
+
+			return task;
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public T getTask(ExecutionAttemptID executionAttemptID) {
+		TaskSlotMapping<T> taskSlotMapping = taskSlotMappings.get(executionAttemptID);
+
+		if (taskSlotMapping != null) {
+			return taskSlotMapping.getTask();
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public Iterator<T> getTasks(JobID jobId) {
+		return new TaskIterator(jobId);
+	}
+
+	@Override
+	public AllocationID getCurrentAllocation(int index) {
+		TaskSlot<T> taskSlot = taskSlots.get(index);
+		if (taskSlot == null) {
+			return null;
+		}
+		return taskSlot.getAllocationId();
+	}
+
+	@Override
+	public MemoryManager getTaskMemoryManager(AllocationID allocationID) throws SlotNotFoundException {
+		TaskSlot<T> taskSlot = getTaskSlot(allocationID);
+		if (taskSlot != null) {
+			return taskSlot.getMemoryManager();
+		} else {
+			throw new SlotNotFoundException(allocationID);
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// TimeoutListener methods
+	// ---------------------------------------------------------------------
+
+	@Override
+	public void notifyTimeout(AllocationID key, UUID ticket) {
+		checkStarted();
+
+		if (slotActions != null) {
+			slotActions.timeoutSlot(key, ticket);
+		}
+	}
+
+	// ---------------------------------------------------------------------
+	// Internal methods
+	// ---------------------------------------------------------------------
+
+	@Nullable
+	private TaskSlot<T> getTaskSlot(AllocationID allocationId) {
+		Preconditions.checkNotNull(allocationId);
+
+		return allocatedSlots.get(allocationId);
+	}
+
+	private void checkRunning() {
+		Preconditions.checkState(
+			state == State.RUNNING,
+			"The %s has to be running.", TaskSlotTableImpl.class.getSimpleName());
+	}
+
+	private void checkStarted() {
+		Preconditions.checkState(
+			state != State.CREATED,
+			"The %s has to be started (not created).", TaskSlotTableImpl.class.getSimpleName());
+	}
+
+	private void checkRunningOrClosing() {
+		Preconditions.checkState(
+			state == State.RUNNING || state == State.CLOSING,
+			"The %s has to be running or being closed.", TaskSlotTableImpl.class.getSimpleName());
+	}
 
 Review comment:
   I think this method is no longer needed.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/145789064 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/145789064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368068307
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -686,7 +696,7 @@ public MemoryManager getTaskMemoryManager(AllocationID allocationID) throws Slot
 
 	@Override
 	public void notifyTimeout(AllocationID key, UUID ticket) {
-		checkInit();
+		checkStartedOrClosing();
 
 Review comment:
   I am not sure whether this always holds true. I could imagine the situation in which the `Timeout` in `TimerService` fires and just before it calls `notifyTimeout`, we finish closing the `TaskSlotTable`. In this situation we would fail this check.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r368061316
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
 ##########
 @@ -273,9 +271,37 @@ public String toString() {
 	}
 
 	@Override
-	public void close() {
-		verifyMemoryFreed();
-		this.memoryManager.shutdown();
+	public CompletableFuture<Void> closeAsync() {
+		return closeAsync(new FlinkException("Closing the slot"));
+	}
+
+	/**
+	 * Close the task slot asynchronously.
+	 *
+	 * <p>Slot is moved to {@link TaskSlotState#RELEASING} state and only once.
+	 * If there are active tasks running in the slot then they are failed.
+	 * The future of all tasks terminated and slot cleaned up is initiated only once and always returned
+	 * in case of multiple attempts to close the slot.
+	 *
+	 * @param cause cause of closing
+	 * @return future of all running task if any being done and slot cleaned up.
+	 */
+	public CompletableFuture<Void> closeAsync(Throwable cause) {
 
 Review comment:
   I would not make this method public and instead leave it package private.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145789064 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145789064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587) 
   * 8c7c178603caf99a23ed9b92f07527d8e5f7169c 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] azagrebin edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-576353246
 
 
   Thanks for the review @tillrohrmann 
   I addressed the comments.
   
   I think the safest way to handle timer service concurrency issues is to schedule its stopping to the main thread of task executor. Then there is only a thread-confined access to its internal container for registered callbacks. I updated PR accordingly.
   
   In long term, we should try to make `Task#notifyFinalState` synced with its terminal future because  internal state of slot and table is managed in main thread inside task executor from `notifyFinalState` and the introduced closing is resulted from terminal futures. This leads to some table methods being called after the closed state is reached. This does not look as an immediate problem atm but complicates judging about the components. The same applies to the mentioned separation of responsibilities: TE -> table -> slot -> task.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#discussion_r362836388
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
 ##########
 @@ -146,12 +149,7 @@ public void start(SlotActions initialSlotActions) {
 	public void stop() {
 		started = false;
 		timerService.stop();
-		allocatedSlots
-			.values()
-			.stream()
-			.filter(slot -> !taskSlots.containsKey(slot.getIndex()))
-			.forEach(TaskSlot::close);
-		allocatedSlots.clear();
+		Preconditions.checkState(allocatedSlots.isEmpty(), "All slots have to be freed before stopping TaskSlotTable");
 
 Review comment:
   Here again, `stop` requires us to first clean up all `allocatedSlots`. I think this is not a good contract. Instead we should think about letting `TaskSlotTable` implement `AutoCloseableAsync` and to trigger the release of the `TaskSlots` when `stop` is being called.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145789064 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145909543 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145789064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587) 
   * 8c7c178603caf99a23ed9b92f07527d8e5f7169c Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145909543) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597) 
   
   <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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-568850864
 
 
   <!--
   Meta data
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142305023 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:a7411a5d1a86381078670898c1f20be4fe3a88f4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903 TriggerType:PUSH TriggerID:a7411a5d1a86381078670898c1f20be4fe3a88f4
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143770612 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:178357b40789d132e1db954a199bf1c62efff474 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240 TriggerType:PUSH TriggerID:178357b40789d132e1db954a199bf1c62efff474
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144064773 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271 TriggerType:PUSH TriggerID:8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144068497 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272 TriggerType:PUSH TriggerID:5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144075835 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:e3be286cd26d25b91e5805eed149bdf5a39aa292 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275 TriggerType:PUSH TriggerID:e3be286cd26d25b91e5805eed149bdf5a39aa292
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144134993 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:d9b76b1b111564fa5aed4a37888f0ec1001f895d Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291 TriggerType:PUSH TriggerID:d9b76b1b111564fa5aed4a37888f0ec1001f895d
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:6b148989636a94325bd5447e71e2d77009e4b4a0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/144154918 TriggerType:PUSH TriggerID:6b148989636a94325bd5447e71e2d77009e4b4a0
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:5b587994ceb858262a96879d490f5f4ed36db4b3 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144163916 TriggerType:PUSH TriggerID:5b587994ceb858262a96879d490f5f4ed36db4b3
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145244518 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:2dda5f2daad2471d6ef07495dc3122be2fe18c76 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510 TriggerType:PUSH TriggerID:2dda5f2daad2471d6ef07495dc3122be2fe18c76
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145739631 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:40e7baea03f7e902574babdd3d9e08641394061d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570 TriggerType:PUSH TriggerID:40e7baea03f7e902574babdd3d9e08641394061d
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/145749292 TriggerType:PUSH TriggerID:139abc989ba80dbfa2265cbd694b45aadcaf6ab9
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145769062 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:3c27130ca6e18c1016f82c84e382208044597509 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575 TriggerType:PUSH TriggerID:3c27130ca6e18c1016f82c84e382208044597509
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/145775093 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578 TriggerType:PUSH TriggerID:618fee6e31b8d8d9c4629aab5f7d70977927c7b1
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145789064 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:11cd349db549a842ba042860e629342f86912e7f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587 TriggerType:PUSH TriggerID:11cd349db549a842ba042860e629342f86912e7f
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/145909543 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   Hash:8c7c178603caf99a23ed9b92f07527d8e5f7169c Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597 TriggerType:PUSH TriggerID:8c7c178603caf99a23ed9b92f07527d8e5f7169c
   -->
   ## CI report:
   
   * a7411a5d1a86381078670898c1f20be4fe3a88f4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142305023) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3903) 
   * 178357b40789d132e1db954a199bf1c62efff474 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143770612) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4240) 
   * 8d2db0ad7b2ce3c268cd8c52a6332a2bc86aca61 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144064773) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4271) 
   * 5625904d8f9e74bb7a7f1ff392094a8c1ea9b9c2 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144068497) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4272) 
   * e3be286cd26d25b91e5805eed149bdf5a39aa292 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144075835) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4275) 
   * d9b76b1b111564fa5aed4a37888f0ec1001f895d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144134993) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4291) 
   * 6b148989636a94325bd5447e71e2d77009e4b4a0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/144154918) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4295) 
   * 5b587994ceb858262a96879d490f5f4ed36db4b3 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144163916) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4298) 
   * 2dda5f2daad2471d6ef07495dc3122be2fe18c76 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145244518) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4510) 
   * 40e7baea03f7e902574babdd3d9e08641394061d Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145739631) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4570) 
   * 139abc989ba80dbfa2265cbd694b45aadcaf6ab9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/145749292) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4572) 
   * 3c27130ca6e18c1016f82c84e382208044597509 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145769062) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4575) 
   * 618fee6e31b8d8d9c4629aab5f7d70977927c7b1 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/145775093) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4578) 
   * 11cd349db549a842ba042860e629342f86912e7f Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145789064) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4587) 
   * 8c7c178603caf99a23ed9b92f07527d8e5f7169c Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/145909543) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4597) 
   
   <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] azagrebin commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
azagrebin commented on issue #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682#issuecomment-576353246
 
 
   Thanks for the review @tillrohrmann 
   I addressed the comments.
   
   I think the safest way to handle timer service concurrency issues is to schedule its stopping to the main thread of task executor. Then there is only a thread-confined access to its internal container for registered callbacks.
   
   In long term, we should try to make `Task#notifyFinalState` synced with its terminal future because  internal state of slot and table is managed in main thread inside task executor from `notifyFinalState` and the introduced closing is resulted from terminal futures. This leads to some table methods being called after the closed state is reached. This does not look as an immediate problem atm but complicates judging about the components. The same applies to the mentioned separation of responsibilities: TE -> table -> slot -> task.

----------------------------------------------------------------
This is an automated message from the 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 #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping

Posted by GitBox <gi...@apache.org>.
tillrohrmann closed pull request #10682: [FLINK-15247][Runtime] Wait for all slots to be free before task executor services shutdown upon stopping
URL: https://github.com/apache/flink/pull/10682
 
 
   

----------------------------------------------------------------
This is an automated message from the 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