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 2018/10/18 09:34:57 UTC

[GitHub] tillrohrmann closed pull request #6867: [FLINK-10580] Harden BootstrapTool#startActorSystem

tillrohrmann closed pull request #6867: [FLINK-10580] Harden BootstrapTool#startActorSystem
URL: https://github.com/apache/flink/pull/6867
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
index 00b61737d20..31f15a9ebb5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
@@ -54,7 +54,6 @@
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.BindException;
-import java.net.ServerSocket;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -156,19 +155,7 @@ public static ActorSystem startActorSystem(
 		}
 
 		while (portsIterator.hasNext()) {
-			// first, we check if the port is available by opening a socket
-			// if the actor system fails to start on the port, we try further
-			ServerSocket availableSocket = NetUtils.createSocketFromPorts(portsIterator, ServerSocket::new);
-
-			int port;
-			if (availableSocket == null) {
-				throw new BindException("Unable to allocate further port in port range: " + portRangeDefinition);
-			} else {
-				port = availableSocket.getLocalPort();
-				try {
-					availableSocket.close();
-				} catch (IOException ignored) {}
-			}
+			final int port = portsIterator.next();
 
 			try {
 				return startActorSystem(
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java
index d1f32cf514a..c01f1aa19fe 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java
@@ -21,10 +21,26 @@
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.CheckedSupplier;
 
+import akka.actor.ActorSystem;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -33,7 +49,9 @@
 /**
  * Tests for {@link BootstrapToolsTest}.
  */
-public class BootstrapToolsTest {
+public class BootstrapToolsTest extends TestLogger {
+
+	private static final Logger LOG = LoggerFactory.getLogger(BootstrapToolsTest.class);
 
 	@Test
 	public void testSubstituteConfigKey() {
@@ -305,4 +323,41 @@ public void testShouldNotUpdateTmpDirectoriesInConfigurationIfNoValueConfigured(
 		BootstrapTools.updateTmpDirectoriesInConfiguration(config, null);
 		assertEquals(config.getString(CoreOptions.TMP_DIRS), CoreOptions.TMP_DIRS.defaultValue());
 	}
+
+	/**
+	 * Tests that we can concurrently create two {@link ActorSystem} without port conflicts.
+	 * This effectively tests that we don't open a socket to check for a ports availability.
+	 * See FLINK-10580 for more details.
+	 */
+	@Test
+	public void testConcurrentActorSystemCreation() throws Exception {
+		final int concurrentCreations = 10;
+		final ExecutorService executorService = Executors.newFixedThreadPool(concurrentCreations);
+		final CyclicBarrier cyclicBarrier = new CyclicBarrier(concurrentCreations);
+
+		try {
+			final List<CompletableFuture<Void>> actorSystemFutures = IntStream.range(0, concurrentCreations)
+				.mapToObj(
+					ignored ->
+						CompletableFuture.supplyAsync(
+							CheckedSupplier.unchecked(() -> {
+								cyclicBarrier.await();
+
+								return BootstrapTools.startActorSystem(
+									new Configuration(),
+									"localhost",
+									"0",
+									LOG);
+							})))
+				.map(
+					// terminate ActorSystems
+					actorSystemFuture ->
+						actorSystemFuture.thenCompose(AkkaUtils::terminateActorSystem)
+				).collect(Collectors.toList());
+
+			FutureUtils.completeAll(actorSystemFutures).get();
+		} finally {
+			ExecutorUtils.gracefulShutdown(10000L, TimeUnit.MILLISECONDS, executorService);
+		}
+	}
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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