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 2020/01/01 15:45:16 UTC

[GitHub] [flink] gaoyunhaii opened a new pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

gaoyunhaii opened a new pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736
 
 
   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Travis CI to do that following [this guide](https://flink.apache.org/contributing/contribute-code.html#open-a-pull-request).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   When shutting down the cluster in standalone mode, the task manager is shutdown by emit SIG_TERM signal. In this case, the directories are cleaned up by the shutdown hooks. However, `NettyShuffleEnvironment` does not register shutdown hook normally, which causes the `flink-netty-shuffle-*` directories not  cleaned finally.
   
   To fix this issue, we registered the corresponding shutdown hooks. To ensure the directories could be removed, the hook must be registered before creating the directories. Therefore, it is not suitable to register shutdown hook in `NettyShuffleEnvironment`. Instead, we chose to allow users to register shutdown hook for each `FileChannelManagerImpl`.   
   
   ## Brief change log
   
   - 1a6b51a57cbf23b054b3b20dd5dbb30a03b561ef registered shutdown hook for `FileChannelManagerImpl` used in `NettyShuffleEnvironment`. 
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - Added test that validates that `flink-netty-shuffle-*` get cleared after task manager received SIG_TERM signals. 
   
   ## 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: **no**
     - 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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368619543
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
 
 Review comment:
   Agree with that, and the code has been modified accordingly.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r373960381
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,180 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	/**
+	 * Marker file indicating the test process is ready to be killed. We could not simply kill the process
+	 * after FileChannelManager has created temporary files since we also need to ensure the caller has
+	 * also registered the shutdown hook if callerHasHook is true.
+	 */
+	private static final String SIGNAL_FILE_FOR_KILLING = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+
+		File signalDir = temporaryFolder.newFolder();
+		File signalFile = new File(FilenameUtils.concat(signalDir.getAbsolutePath(), SIGNAL_FILE_FOR_KILLING));
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			signalFile.getAbsolutePath());
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till the process has created temporary files and registered the corresponding shutdown hooks.
+			TestJvmProcess.waitForMarkerFile(signalFile, TEST_TIMEOUT.toMillis());
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			assertFalse("The file channel manager test process does not terminate in time, its output is: \n"
+						+ fileChannelManagerTestProcess.getProcessOutput(),
+					fileChannelManagerTestProcess.isAlive());
+
+			// Checks if the directories are cleared.
+			assertFalse("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX));
+		} finally {
+			fileChannelManagerTestProcess.destroy();
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
 
 Review comment:
   Have modified accordingly.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   * a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368880380
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			fileChannelManagerTestProcess.destroy();
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				// Verifies the case that both FileChannelManager and its upper component
+				// have registered shutdown hooks, like in IOManager.
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Singles main process what we can be killed.
 
 Review comment:
   The comments are modified

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r373960409
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -99,10 +126,17 @@ public Enumerator createChannelEnumerator() {
 	 */
 	@Override
 	public void close() throws Exception {
+		// Marks shut down and exit if it has already shut down.
 
 Review comment:
   Have modified accordingly.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   
   <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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r365642499
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -51,12 +52,30 @@
 	/** The number of the next path to use. */
 	private volatile int nextPath;
 
+	private final String prefix;
+
+	private final Thread shutdownHook;
+
 	public FileChannelManagerImpl(String[] tempDirs, String prefix) {
+		this(tempDirs, prefix, false);
+	}
+
+	public FileChannelManagerImpl(String[] tempDirs, String prefix, boolean deleteOnShutdown) {
 		checkNotNull(tempDirs, "The temporary directories must not be null.");
 		checkArgument(tempDirs.length > 0, "The temporary directories must not be empty.");
 
 		this.random = new Random();
 		this.nextPath = 0;
+		this.prefix = prefix;
+
+		if (deleteOnShutdown) {
+			shutdownHook = ShutdownHookUtil.addShutdownHook(this, String.format("%s-%s", getClass().getSimpleName(), prefix), LOG);
 
 Review comment:
   I got the point now. 
   
   I think it is better to make `FileChannelManagerImpl` always have the `shutdownHook`, and then it does not need judge whether it is `null` in other usages. Also it does not need to care about this argument `deleteOnShutdown` to true or false for upper component. 
   
   In order to avoid repeated `close` calling by `IOManager` usage, we can add the atomic boolean var in `FileChannelManagerImpl` and check this state in other usages as well.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r373960398
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,180 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	/**
+	 * Marker file indicating the test process is ready to be killed. We could not simply kill the process
+	 * after FileChannelManager has created temporary files since we also need to ensure the caller has
+	 * also registered the shutdown hook if callerHasHook is true.
 
 Review comment:
   Have modified accordingly.

----------------------------------------------------------------
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] gaoyunhaii edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-572362847
 
 
   Hi @zhijiangW , very thanks for the review! The reason that we cannot directly add shutdown hook to `NettyShuffleEnvironment`,  it cannot ensure the directories get cleaned up in all cases, since the directories are created in the constructor of `FileChannelManagerImpl`, which comes before registering  shutdown hook in `NettyShuffleEnvironment's` constructor. If task  managers receive SIG_TERM between the two actions, the directories will not be cleaned. Currently `IOManager` also has this problem.
   
   Therefore, it should be better to directly register shutdown hook for `FileChannelManagerImpl`, before the directories are actually created.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368636592
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Single main process what we can be killed.
+			new File(couldKillSignalFilePath).createNewFile();
+
+			// Waits till get killed. If we have not killed in time, make sure we exit finally.
+			// Meanwhile, the test will fail due to process not terminated in time.
+			Thread.sleep(5 * TEST_TIMEOUT.toMillis());
 
 Review comment:
   I think the waiting test process would wait enough time before the `kill` process takes effect, otherwise we may have unstable scenarios. If the killing process cannot kill the process in `TEST_TIMEOUT`, then the test would fail directly with the timeout logic in the test function, and it will not waiting for `5 * TEST_TIMEOUT`. 

----------------------------------------------------------------
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] gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-576593784
 
 
   Hi @zhijiangW very thanks for the review and I have updated the PR. 

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368794377
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
 
 Review comment:
   COULD_KILL_SIGNAL_FILE -> SIGNAL_FILE_FOR_KILLING, and give some comments for what it is used for.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368906400
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -99,10 +126,17 @@ public Enumerator createChannelEnumerator() {
 	 */
 	@Override
 	public void close() throws Exception {
+		// Marks shut down and exit if it has already shut down.
 
 Review comment:
   exit -> exits

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368878606
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
 
 Review comment:
   Agree with that, and I have modified the code accordingly

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   
   <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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   
   <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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368878882
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
 
 Review comment:
   I have modified the code accordingly.

----------------------------------------------------------------
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] gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-572377049
 
 
   @flinkbot run travis

----------------------------------------------------------------
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] zhijiangW commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-571553868
 
 
   Thanks for this fix @gaoyunhaii !
   
   The `FileChannelManagerImpl` would be used by both `IOManager` and `NettyShuffleEnvironment`. The `IOManager` already added the shutdown hook to close the files finally. I am wondering why can not add the shutdown hook to the `NettyShuffleEnvironment` instead of `FileChannelManagerImpl`?

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368878781
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
 
 Review comment:
   I have modified the code accordingly.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368918171
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,180 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	/**
+	 * Marker file indicating the test process is ready to be killed. We could not simply kill the process
+	 * after FileChannelManager has created temporary files since we also need to ensure the caller has
+	 * also registered the shutdown hook if callerHasHook is true.
+	 */
+	private static final String SIGNAL_FILE_FOR_KILLING = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+
+		File signalDir = temporaryFolder.newFolder();
+		File signalFile = new File(FilenameUtils.concat(signalDir.getAbsolutePath(), SIGNAL_FILE_FOR_KILLING));
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			signalFile.getAbsolutePath());
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till the process has created temporary files and registered the corresponding shutdown hooks.
+			TestJvmProcess.waitForMarkerFile(signalFile, TEST_TIMEOUT.toMillis());
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			assertFalse("The file channel manager test process does not terminate in time, its output is: \n"
+						+ fileChannelManagerTestProcess.getProcessOutput(),
+					fileChannelManagerTestProcess.isAlive());
+
+			// Checks if the directories are cleared.
+			assertFalse("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX));
+		} finally {
+			fileChannelManagerTestProcess.destroy();
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
 
 Review comment:
   couldKillSignalFilePath -> signalFilePath

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368471642
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
 
 Review comment:
   I guess this part is reductant because the `javaCommand` is actually not used below?

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368787114
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -51,12 +54,27 @@
 	/** The number of the next path to use. */
 	private volatile int nextPath;
 
+	/** Prefix of the temporary directories to create */
+	private final String prefix;
+
+	/** Flag to signify that the file channel manager has been shut down already. */
+	private final AtomicBoolean isShutdown = new AtomicBoolean();
 
 Review comment:
   I think we should give some comments for the reason of atomic here, which is for the case of multiple shutdown hooks.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368487987
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
 
 Review comment:
   It is better to give some comment explanations here why we need `callerHasHook` for testing. E.g. we want to verify the case that both the upper component and its internal `FileChannelManagerImpl` have the shutdown hook in practice like `IOManager`.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368467392
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
 
 Review comment:
   Extend `TestLogger` for better tracking.

----------------------------------------------------------------
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] gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-572362847
 
 
   Hi @zhijiangW , very thanks for the review. The reason that we cannot directly add shutdown hook to `NettyShuffleEnvironment`,  it cannot ensure the directories get cleaned up in all cases, since the directories are created in the constructor of `FileChannelManagerImpl`, which comes before registering  shutdown hook in `NettyShuffleEnvironment's` constructor. If task  managers receive SIG_TERM between the two actions, the directories will not be cleaned. Currently `IOManager` also has this problem.
   
   Therefore, it should be better to directly register shutdown hook for `FileChannelManagerImpl`, before the directories are actually created.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   
   <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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   
   <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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368791957
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
 
 Review comment:
   we can also remove this part if making use of `TestJvmProcess.waitForMarkerFile()` as mentioned above.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r366133442
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
 
 Review comment:
   I think `assumeTrue` would skip the tests if conditions not satisfied, it will not cause the test fails.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147213597 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   * a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147213597) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788) 
   
   <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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368474123
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
 
 Review comment:
   Make this creation out of `try` clause, then we do not need to check whether it is null in `finally` clause.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368872499
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -51,12 +54,27 @@
 	/** The number of the next path to use. */
 	private volatile int nextPath;
 
+	/** Prefix of the temporary directories to create */
+	private final String prefix;
+
+	/** Flag to signify that the file channel manager has been shut down already. */
+	private final AtomicBoolean isShutdown = new AtomicBoolean();
 
 Review comment:
   I have added the 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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368889978
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			fileChannelManagerTestProcess.destroy();
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
 
 Review comment:
   I am a little worry since that the class is used as an entry point class of one process. Although it seems  not cause error with the current JDK, I am afraid it may cause unstableness in 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368794784
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			fileChannelManagerTestProcess.destroy();
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				// Verifies the case that both FileChannelManager and its upper component
+				// have registered shutdown hooks, like in IOManager.
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Singles main process what we can be killed.
 
 Review comment:
   Singles -> Signals

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368795333
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			fileChannelManagerTestProcess.destroy();
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
 
 Review comment:
   public -> 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368794784
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			fileChannelManagerTestProcess.destroy();
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				// Verifies the case that both FileChannelManager and its upper component
+				// have registered shutdown hooks, like in IOManager.
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Singles main process what we can be killed.
 
 Review comment:
   Singles -> Signals
   Signals the main process to execute the kill action.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368632832
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
 
 Review comment:
   Agree with that and have modified the code accordingly.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368618365
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
 
 Review comment:
   The `javaCommand` is truly redundant and I have removed the related code. 

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r365663234
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File confDir = temporaryFolder.newFolder();
+		File confFile = new File(confDir + "/flink-conf.yaml");
+
+		File taskManagerTmpDir = temporaryFolder.newFolder();
+
+		Configuration config = new Configuration();
+		config.setString(JobManagerOptions.ADDRESS, "localhost");
+		config.setString(RestOptions.BIND_PORT, "0");
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString());
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath());
+		config.setString(CoreOptions.TMP_DIRS, taskManagerTmpDir.getAbsolutePath());
+		config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m");
+		config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100);
+		config.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, "512m");
+
+		try (FileOutputStream fos = new FileOutputStream(confFile);
+			BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos))) {
+			for (Map.Entry<String, String> e : config.toMap().entrySet()) {
+				writer.write(e.getKey());
+				writer.write(": ");
+				writer.write(e.getValue());
+				writer.newLine();
+			}
+
+			writer.flush();
+		}
+
+		TaskManagerProcess taskManagerProcess = null;
+
+		try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) {
 
 Review comment:
   It is too heavy-height to start the cluster besides the preparation of above configuration and writer. We can construct the `FileChannelManagerImpl` via `TestJvmProcess` directly for simple. Actually we only want to verify that the new added shutdown hook to `FileChannelManagerImpl` work correctly, no matter which component covers 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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   
   <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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368791624
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
 
 Review comment:
   Also we could make use of  existing `TestJvmProcess.waitForMarkerFile()` instead if we pass the signal file in `FileChannelManagerTestProcess`

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147213597 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788 TriggerType:MANUAL TriggerID:581694694
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147213597 TriggerType:MANUAL TriggerID:581694694
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   * a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147213597) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788) 
   
   <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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368619543
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
 
 Review comment:
   Agree with that we could change to package private, and the code has been modified accordingly.
   
   The exception should not be able to be removed, since the super constructor throws 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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147213597 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788 TriggerType:MANUAL TriggerID:581694694
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   * a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147213597) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788) 
   
   <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] gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-576352767
 
 
   Hi @zhijiangW Very thanks for the review, and I have updated the PR accordingly. :)

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368406546
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
 
 Review comment:
   Sorry I read it wrong as `assertTrue`. Ignore my previous comment.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368878959
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
 
 Review comment:
   The name has been changed.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   
   <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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368617430
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
 
 Review comment:
   Added the extension accordingly.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r366133522
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File confDir = temporaryFolder.newFolder();
+		File confFile = new File(confDir + "/flink-conf.yaml");
+
+		File taskManagerTmpDir = temporaryFolder.newFolder();
+
+		Configuration config = new Configuration();
+		config.setString(JobManagerOptions.ADDRESS, "localhost");
+		config.setString(RestOptions.BIND_PORT, "0");
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString());
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath());
+		config.setString(CoreOptions.TMP_DIRS, taskManagerTmpDir.getAbsolutePath());
+		config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m");
+		config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100);
+		config.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, "512m");
+
+		try (FileOutputStream fos = new FileOutputStream(confFile);
+			BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos))) {
+			for (Map.Entry<String, String> e : config.toMap().entrySet()) {
+				writer.write(e.getKey());
+				writer.write(": ");
+				writer.write(e.getValue());
+				writer.newLine();
+			}
+
+			writer.flush();
+		}
+
+		TaskManagerProcess taskManagerProcess = null;
+
+		try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) {
 
 Review comment:
   Agree with that and I have modified the test accordingly. 

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368467950
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
 
 Review comment:
   better to add an empty line before this import.

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368616401
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -99,10 +123,17 @@ public Enumerator createChannelEnumerator() {
 	 */
 	@Override
 	public void close() throws Exception {
+		// Marks shut down and exit if it already was shut down.
 
 Review comment:
   I have modified the 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368489911
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
 
 Review comment:
   can be package private and remove the `throws 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r363711985
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -51,12 +52,30 @@
 	/** The number of the next path to use. */
 	private volatile int nextPath;
 
+	private final String prefix;
+
+	private final Thread shutdownHook;
+
 	public FileChannelManagerImpl(String[] tempDirs, String prefix) {
+		this(tempDirs, prefix, false);
+	}
+
+	public FileChannelManagerImpl(String[] tempDirs, String prefix, boolean deleteOnShutdown) {
 		checkNotNull(tempDirs, "The temporary directories must not be null.");
 		checkArgument(tempDirs.length > 0, "The temporary directories must not be empty.");
 
 		this.random = new Random();
 		this.nextPath = 0;
+		this.prefix = prefix;
+
+		if (deleteOnShutdown) {
+			shutdownHook = ShutdownHookUtil.addShutdownHook(this, String.format("%s-%s", getClass().getSimpleName(), prefix), LOG);
 
 Review comment:
   Can we always add the shutdown hook to remove the argument `deleteOnShutdown`?

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r366134565
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File confDir = temporaryFolder.newFolder();
+		File confFile = new File(confDir + "/flink-conf.yaml");
+
+		File taskManagerTmpDir = temporaryFolder.newFolder();
+
+		Configuration config = new Configuration();
+		config.setString(JobManagerOptions.ADDRESS, "localhost");
+		config.setString(RestOptions.BIND_PORT, "0");
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString());
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath());
+		config.setString(CoreOptions.TMP_DIRS, taskManagerTmpDir.getAbsolutePath());
+		config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m");
+		config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100);
+		config.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, "512m");
+
+		try (FileOutputStream fos = new FileOutputStream(confFile);
+			BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos))) {
+			for (Map.Entry<String, String> e : config.toMap().entrySet()) {
+				writer.write(e.getKey());
+				writer.write(": ");
+				writer.write(e.getValue());
+				writer.newLine();
+			}
+
+			writer.flush();
+		}
+
+		TaskManagerProcess taskManagerProcess = null;
+
+		try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) {
+			String javaCommand = getJavaCommandPath();
+			if (javaCommand == null) {
+				fail("Could not find java executable.");
+			}
+
+			clusterEntrypoint.startCluster();
+
+			taskManagerProcess = new TaskManagerProcess(javaCommand, confDir.getAbsolutePath());
+			taskManagerProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+			File[] nettyShuffleTmpFiles = findNettyShuffleDirectories(taskManagerTmpDir);
+
+			while (nettyShuffleTmpFiles.length == 0 && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+				nettyShuffleTmpFiles = findNettyShuffleDirectories(taskManagerTmpDir);
+			}
+
+			if (nettyShuffleTmpFiles.length == 0) {
+				fail("The TaskManager process does not create shuffle directories in time, " +
+						"its output is: \n" + taskManagerProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + taskManagerProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (taskManagerProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (taskManagerProcess.isAlive()) {
+				fail("The TaskManager process does not terminate in time, its output is: \n" + taskManagerProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			nettyShuffleTmpFiles = findNettyShuffleDirectories(taskManagerTmpDir);
+			assertEquals("The TaskManager does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + taskManagerProcess.getProcessOutput(),
+					0, nettyShuffleTmpFiles.length);
+		}
 
 Review comment:
   Agree with that we should add the finally block, and destroy the process in the finally block. Besides, I think we should not need to delete files explicitly since we are using the `TemporaryFolder` and it should be able to ensure files get cleared after 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368915715
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,180 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	/**
+	 * Marker file indicating the test process is ready to be killed. We could not simply kill the process
+	 * after FileChannelManager has created temporary files since we also need to ensure the caller has
+	 * also registered the shutdown hook if callerHasHook is true.
 
 Review comment:
   <tt> callerHasHook </tt>

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368637168
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Single main process what we can be killed.
+			new File(couldKillSignalFilePath).createNewFile();
+
+			// Waits till get killed. If we have not killed in time, make sure we exit finally.
+			// Meanwhile, the test will fail due to process not terminated in time.
+			Thread.sleep(5 * TEST_TIMEOUT.toMillis());
+			System.exit(1);
 
 Review comment:
   I think Similar to the last issue, I think the test would be able to fail directly after `TEST_TIMEOUT` due to the test process does not get killed in time, as specified in the test function. 

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   
   <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] gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-581694694
 
 
   @flinkbot run azure

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   
   <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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r365663716
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File confDir = temporaryFolder.newFolder();
+		File confFile = new File(confDir + "/flink-conf.yaml");
+
+		File taskManagerTmpDir = temporaryFolder.newFolder();
+
+		Configuration config = new Configuration();
+		config.setString(JobManagerOptions.ADDRESS, "localhost");
+		config.setString(RestOptions.BIND_PORT, "0");
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString());
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath());
+		config.setString(CoreOptions.TMP_DIRS, taskManagerTmpDir.getAbsolutePath());
+		config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m");
+		config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100);
+		config.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, "512m");
+
+		try (FileOutputStream fos = new FileOutputStream(confFile);
+			BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos))) {
+			for (Map.Entry<String, String> e : config.toMap().entrySet()) {
+				writer.write(e.getKey());
+				writer.write(": ");
+				writer.write(e.getValue());
+				writer.newLine();
+			}
+
+			writer.flush();
+		}
+
+		TaskManagerProcess taskManagerProcess = null;
+
+		try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) {
 
 Review comment:
   In addition, we should rename `NettyShuffleEnvironmentCleanupTest` to `FileChannelManagerImplTest` instead.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368902755
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -51,12 +54,27 @@
 	/** The number of the next path to use. */
 	private volatile int nextPath;
 
+	/** Prefix of the temporary directories to create */
+	private final String prefix;
+
+	/** Flag to signify that the file channel manager has been shut down already. */
+	private final AtomicBoolean isShutdown = new AtomicBoolean();
 
 Review comment:
   typo signify

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/147213597 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   Hash:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788 TriggerType:PUSH TriggerID:a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   * a20895d3ac0e5fd3eb1f4fd3c665519bc15ae803 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/147213597) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4788) 
   
   <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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   
   <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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368617580
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
 
 Review comment:
   Added the empty line.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   
   <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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368878847
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
 
 Review comment:
   I have modified the code accordingly.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368798136
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Single main process what we can be killed.
+			new File(couldKillSignalFilePath).createNewFile();
+
+			// Waits till get killed. If we have not killed in time, make sure we exit finally.
+			// Meanwhile, the test will fail due to process not terminated in time.
+			Thread.sleep(5 * TEST_TIMEOUT.toMillis());
 
 Review comment:
   The normal case should take less than several seconds, so 30s should be enough for bearing the 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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368788718
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
 
 Review comment:
   Can we check via `fileChannelDir.list().length == 0` to avoid relying on `COULD_KILL_SIGNAL_FILE`? If so we can get ride of it completely and only rely on the original `FileChannelManagerImpl`.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368489680
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Single main process what we can be killed.
+			new File(couldKillSignalFilePath).createNewFile();
+
+			// Waits till get killed. If we have not killed in time, make sure we exit finally.
+			// Meanwhile, the test will fail due to process not terminated in time.
+			Thread.sleep(5 * TEST_TIMEOUT.toMillis());
+			System.exit(1);
 
 Review comment:
   We should make this test fail if entering the `exit(1)`?

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   
   <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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368793697
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
 
 Review comment:
   also use `assertFalse` for simple

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368915715
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,180 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	/**
+	 * Marker file indicating the test process is ready to be killed. We could not simply kill the process
+	 * after FileChannelManager has created temporary files since we also need to ensure the caller has
+	 * also registered the shutdown hook if callerHasHook is true.
 
 Review comment:
   <tt> callerHasHook</tt>

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r364544133
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -51,12 +52,30 @@
 	/** The number of the next path to use. */
 	private volatile int nextPath;
 
+	private final String prefix;
+
+	private final Thread shutdownHook;
+
 	public FileChannelManagerImpl(String[] tempDirs, String prefix) {
+		this(tempDirs, prefix, false);
+	}
+
+	public FileChannelManagerImpl(String[] tempDirs, String prefix, boolean deleteOnShutdown) {
 		checkNotNull(tempDirs, "The temporary directories must not be null.");
 		checkArgument(tempDirs.length > 0, "The temporary directories must not be empty.");
 
 		this.random = new Random();
 		this.nextPath = 0;
+		this.prefix = prefix;
+
+		if (deleteOnShutdown) {
+			shutdownHook = ShutdownHookUtil.addShutdownHook(this, String.format("%s-%s", getClass().getSimpleName(), prefix), LOG);
 
 Review comment:
   This is due to in this PR I think it should be better to not affect the current behaviors of `IOManager`.  Although `IOManager` also have the problem that register shutdown hook after the directories are created, I think it might be better to fix it in separate PR if we agree with how to deal with it. Since `IOManager` already has a shutdown hook, for `IOManager` we also need to allow the `FileChannelManagerImpl#close` be closed for multiple times (i.e., using `closed` flag to skip cleanup in repeat `close` calls). 

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r365663503
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File confDir = temporaryFolder.newFolder();
+		File confFile = new File(confDir + "/flink-conf.yaml");
+
+		File taskManagerTmpDir = temporaryFolder.newFolder();
+
+		Configuration config = new Configuration();
+		config.setString(JobManagerOptions.ADDRESS, "localhost");
+		config.setString(RestOptions.BIND_PORT, "0");
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString());
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath());
+		config.setString(CoreOptions.TMP_DIRS, taskManagerTmpDir.getAbsolutePath());
+		config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m");
+		config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100);
+		config.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, "512m");
+
+		try (FileOutputStream fos = new FileOutputStream(confFile);
+			BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos))) {
+			for (Map.Entry<String, String> e : config.toMap().entrySet()) {
+				writer.write(e.getKey());
+				writer.write(": ");
+				writer.write(e.getValue());
+				writer.newLine();
+			}
+
+			writer.flush();
+		}
+
+		TaskManagerProcess taskManagerProcess = null;
+
+		try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) {
+			String javaCommand = getJavaCommandPath();
+			if (javaCommand == null) {
+				fail("Could not find java executable.");
+			}
+
+			clusterEntrypoint.startCluster();
+
+			taskManagerProcess = new TaskManagerProcess(javaCommand, confDir.getAbsolutePath());
+			taskManagerProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+			File[] nettyShuffleTmpFiles = findNettyShuffleDirectories(taskManagerTmpDir);
+
+			while (nettyShuffleTmpFiles.length == 0 && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+				nettyShuffleTmpFiles = findNettyShuffleDirectories(taskManagerTmpDir);
+			}
+
+			if (nettyShuffleTmpFiles.length == 0) {
+				fail("The TaskManager process does not create shuffle directories in time, " +
+						"its output is: \n" + taskManagerProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + taskManagerProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (taskManagerProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (taskManagerProcess.isAlive()) {
+				fail("The TaskManager process does not terminate in time, its output is: \n" + taskManagerProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			nettyShuffleTmpFiles = findNettyShuffleDirectories(taskManagerTmpDir);
+			assertEquals("The TaskManager does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + taskManagerProcess.getProcessOutput(),
+					0, nettyShuffleTmpFiles.length);
+		}
 
 Review comment:
   we should add a finally part to cleanup the `FileChannelManagerImpl` as the bottom line. In case this test fails unexpected, we still do not leak any files.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570062911
 
 
   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 1a6b51a57cbf23b054b3b20dd5dbb30a03b561ef (Wed Jan 01 15:48:03 UTC 2020)
   
   **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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368464542
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelManagerImpl.java
 ##########
 @@ -99,10 +123,17 @@ public Enumerator createChannelEnumerator() {
 	 */
 	@Override
 	public void close() throws Exception {
+		// Marks shut down and exit if it already was shut down.
 
 Review comment:
   nit: it already was -> it was already

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e 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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   
   <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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368619284
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
 
 Review comment:
   Agree with that, and the code is modified accordingly.

----------------------------------------------------------------
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] zhijiangW commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-576503887
 
 
   Thanks for the updates @gaoyunhaii ! I left some other comments and I think it is good to merge after addressing them.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368792665
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+			callerHasHook,
+			fileChannelDir.getAbsolutePath(),
+			FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+
+		try {
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
 
 Review comment:
   Use `assertFalse("", fileChannelManagerTestProcess.isAlive())` for simple.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368489397
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,192 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
+
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(false);
+	}
+
+	@Test
+	public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception {
+		testDirectoriesCleanupOnKill(true);
+	}
+
+	private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
+				|| OperatingSystem.isFreeBSD()
+				|| OperatingSystem.isSolaris()
+				|| OperatingSystem.isMac());
+
+		File fileChannelDir = temporaryFolder.newFolder();
+		File signalDir = temporaryFolder.newFolder();
+
+		FileChannelManagerTestProcess fileChannelManagerTestProcess = null;
+
+		String javaCommand = getJavaCommandPath();
+		if (javaCommand == null) {
+			fail("Could not find java executable.");
+		}
+
+		try {
+			fileChannelManagerTestProcess = new FileChannelManagerTestProcess(
+					callerHasHook,
+					fileChannelDir.getAbsolutePath(),
+					FilenameUtils.concat(signalDir.getAbsolutePath(), COULD_KILL_SIGNAL_FILE));
+			fileChannelManagerTestProcess.startProcess();
+
+			// Waits till netty shuffle environment has created the tmp directories.
+			Deadline deadline = Deadline.now().plus(TEST_TIMEOUT);
+
+			while (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (!fileOrDirExists(signalDir, COULD_KILL_SIGNAL_FILE) ||
+					!fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) {
+				fail("The file channel manager test process does not create target directories in time, " +
+						"its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			Process kill = Runtime.getRuntime().exec("kill " + fileChannelManagerTestProcess.getProcessId());
+			kill.waitFor();
+			assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue());
+
+			deadline = Deadline.now().plus(TEST_TIMEOUT);
+			while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) {
+				Thread.sleep(100);
+			}
+
+			if (fileChannelManagerTestProcess.isAlive()) {
+				fail("The file channel manager test process does not terminate in time, its output is: \n" + fileChannelManagerTestProcess.getProcessOutput());
+			}
+
+			// Checks if the directories are cleared.
+			assertThat("The file channel manager test process does not remove the tmp shuffle directories after termination, " +
+							"its output is \n" + fileChannelManagerTestProcess.getProcessOutput(),
+					fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX),
+					is(false));
+		} finally {
+			if (fileChannelManagerTestProcess != null) {
+				fileChannelManagerTestProcess.destroy();
+			}
+		}
+	}
+
+	private boolean fileOrDirExists(File rootTmpDir, String namePattern) {
+		File[] candidates = rootTmpDir.listFiles((dir, name) -> name.contains(namePattern));
+		return candidates != null && candidates.length > 0;
+	}
+
+	/**
+	 * The {@link FileChannelManagerCleanupRunner} instance running in a separate JVM process.
+	 */
+	private static class FileChannelManagerTestProcess extends TestJvmProcess {
+		private final boolean callerHasHook;
+		private final String tmpDirectories;
+		private final String couldKillSignalFilePath;
+
+		public FileChannelManagerTestProcess(boolean callerHasHook, String tmpDirectories, String couldKillSignalFilePath) throws Exception {
+			this.callerHasHook = callerHasHook;
+			this.tmpDirectories = tmpDirectories;
+			this.couldKillSignalFilePath = couldKillSignalFilePath;
+		}
+
+		@Override
+		public String getName() {
+			return "File Channel Manager Test";
+		}
+
+		@Override
+		public String[] getJvmArgs() {
+			return new String[]{
+					Boolean.toString(callerHasHook),
+					tmpDirectories,
+					couldKillSignalFilePath
+			};
+		}
+
+		@Override
+		public String getEntryPointClassName() {
+			return FileChannelManagerCleanupRunner.class.getName();
+		}
+	}
+
+	/**
+	 * The entry point class to test the file channel manager cleanup with shutdown hook.
+	 */
+	public static class FileChannelManagerCleanupRunner {
+
+		public static void main(String[] args) throws Exception{
+			boolean callerHasHook = Boolean.parseBoolean(args[0]);
+			String tmpDirectory = args[1];
+			String couldKillSignalFilePath = args[2];
+
+			FileChannelManager manager = new FileChannelManagerImpl(new String[]{tmpDirectory}, DIR_NAME_PREFIX);
+
+			if (callerHasHook) {
+				ShutdownHookUtil.addShutdownHook(() -> manager.close(), "Caller", LOG);
+			}
+
+			// Single main process what we can be killed.
+			new File(couldKillSignalFilePath).createNewFile();
+
+			// Waits till get killed. If we have not killed in time, make sure we exit finally.
+			// Meanwhile, the test will fail due to process not terminated in time.
+			Thread.sleep(5 * TEST_TIMEOUT.toMillis());
 
 Review comment:
   Can we still refer to the `TEST_TIMEOUT` for waiting before failure? Actually 50s is too long for this test I guess.

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368915715
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,180 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	/**
+	 * Marker file indicating the test process is ready to be killed. We could not simply kill the process
+	 * after FileChannelManager has created temporary files since we also need to ensure the caller has
+	 * also registered the shutdown hook if callerHasHook is true.
 
 Review comment:
   `<tt> callerHasHook </tt>`

----------------------------------------------------------------
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] zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r365662444
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentCleanupTest.java
 ##########
 @@ -0,0 +1,181 @@
+/*
+ * 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.io.network;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.time.Duration;
+import java.util.Map;
+
+import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Verifies whether netty shuffle releases all the resources on shutdown, like the temporary directories.
+ */
+public class NettyShuffleEnvironmentCleanupTest {
+	private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10);
+
+	@Rule
+	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Rule
+	public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource();
+
+	@Test
+	public void testRemovingTmpDirectoriesOnSignals() throws Exception {
+		assumeTrue(OperatingSystem.isLinux()
 
 Review comment:
   If the shutdown hook only supports the following systems, maybe it is better to not fail for other systems, i mean this test can be ignored directly instead of failing.

----------------------------------------------------------------
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 #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-570066854
 
 
   <!--
   Meta data
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142804653 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:4b605068e32d3eb15a51f838ed29918d1224959a Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021 TriggerType:PUSH TriggerID:4b605068e32d3eb15a51f838ed29918d1224959a
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:PUSH TriggerID:9430066683a67318f9685de8a58904972c5dbaca
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142829633 TriggerType:MANUAL TriggerID:572377049
   Hash:9430066683a67318f9685de8a58904972c5dbaca Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026 TriggerType:MANUAL TriggerID:572377049
   Hash:2185007c824d21817356c9dfb9c9e09846e27f7e Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2185007c824d21817356c9dfb9c9e09846e27f7e
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144257501 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:d7ab35b18c5964b837be1d52611623d7c271dc99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316 TriggerType:PUSH TriggerID:d7ab35b18c5964b837be1d52611623d7c271dc99
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145256684 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514 TriggerType:PUSH TriggerID:2000e3bfb01cdb7e9c2fe7c882baad291e098de4
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/145356392 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:3104fe46714388247df22b5372e5c3c134834646 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536 TriggerType:PUSH TriggerID:3104fe46714388247df22b5372e5c3c134834646
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/145405231 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   Hash:d935ac02c5e5f53566225df738862226a7e50aa1 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545 TriggerType:PUSH TriggerID:d935ac02c5e5f53566225df738862226a7e50aa1
   -->
   ## CI report:
   
   * 4b605068e32d3eb15a51f838ed29918d1224959a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142804653) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4021) 
   * 9430066683a67318f9685de8a58904972c5dbaca Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142829633) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4026) 
   * 2185007c824d21817356c9dfb9c9e09846e27f7e UNKNOWN
   * d7ab35b18c5964b837be1d52611623d7c271dc99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144257501) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4316) 
   * 2000e3bfb01cdb7e9c2fe7c882baad291e098de4 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145256684) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4514) 
   * 3104fe46714388247df22b5372e5c3c134834646 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/145356392) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4536) 
   * d935ac02c5e5f53566225df738862226a7e50aa1 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/145405231) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4545) 
   
   <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] gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on issue #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#issuecomment-573978662
 
 
   Hi @zhijiangW very thanks for the review! I have updated the PR according to the 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] zhijiangW merged pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
zhijiangW merged pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736
 
 
   

----------------------------------------------------------------
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] gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #10736: [FLINK-15010][Network] Add shutdown hook to ensure cleanup netty shuffle directories
URL: https://github.com/apache/flink/pull/10736#discussion_r368878959
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java
 ##########
 @@ -0,0 +1,187 @@
+/*
+ * 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.io.disk;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.runtime.testutils.TestJvmProcess;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.time.Duration;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests the logic of {@link FileChannelManagerImpl}.
+ */
+public class FileChannelManagerImplTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class);
+
+	private static final String DIR_NAME_PREFIX = "manager-test";
+
+	private static final String COULD_KILL_SIGNAL_FILE = "could-kill";
 
 Review comment:
   The name has been changed and the comments are added.

----------------------------------------------------------------
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