You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2019/01/24 07:57:04 UTC

[flink] 02/02: [FLINK-11356][tests] Port JobManagerStartupTest to new code base

This is an automated email from the ASF dual-hosted git repository.

trohrmann pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit bb64862b8b7e3ee35625bfe8cb003e8eada8e949
Author: Till Rohrmann <tr...@apache.org>
AuthorDate: Fri Jan 18 15:48:56 2019 +0100

    [FLINK-11356][tests] Port JobManagerStartupTest to new code base
    
    - Moved JobManagerStartupTest#testStartupWithPortInUse to
    BootstrapToolsTest#testActorSystemInstantiationFailureWhenPortOccupied
    
    - Moved JobManagerStartupTest#testJobManagerStartupFails to
    BlobServerTest#testFailureIfStorageDirectoryCannotBeCreated
    
    This closes #7541.
---
 .../apache/flink/runtime/blob/BlobServerTest.java  |  76 +++++++++++
 .../clusterframework/BootstrapToolsTest.java       |  26 ++++
 .../runtime/jobmanager/JobManagerStartupTest.java  | 148 ---------------------
 3 files changed, 102 insertions(+), 148 deletions(-)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerTest.java
new file mode 100644
index 0000000..cc335ca
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerTest.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.blob;
+
+import org.apache.flink.configuration.BlobServerOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.annotation.Nonnull;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeFalse;
+
+/**
+ * Tests for the {@link BlobServer}.
+ */
+public class BlobServerTest extends TestLogger {
+
+	@ClassRule
+	public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	/**
+	 * Tests that the {@link BlobServer} fails if the blob storage directory
+	 * cannot be created.
+	 */
+	@Test
+	public void testFailureIfStorageDirectoryCannotBeCreated() throws IOException {
+		final Configuration configuration = new Configuration();
+		final File blobStorageDirectory = createNonWritableDirectory();
+
+		final String nonExistDirectory = new File(blobStorageDirectory, "does_not_exist_for_sure").getAbsolutePath();
+		configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, nonExistDirectory);
+
+		try (BlobServer ignored = new BlobServer(configuration, new VoidBlobStore())) {
+			fail("Expected that the BlobServer initialization fails.");
+		} catch (IOException expected) {
+			// expected
+		}
+	}
+
+	@Nonnull
+	private File createNonWritableDirectory() throws IOException {
+		assumeFalse(OperatingSystem.isWindows()); //setWritable doesn't work on Windows.
+		final File blobStorageDirectory = temporaryFolder.newFolder();
+		assertTrue(blobStorageDirectory.setExecutable(true, false));
+		assertTrue(blobStorageDirectory.setReadable(true, false));
+		assertTrue(blobStorageDirectory.setWritable(false, false));
+		return blobStorageDirectory;
+	}
+
+}
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 805e970..54aadf5 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
@@ -23,6 +23,7 @@ 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.ExceptionUtils;
 import org.apache.flink.util.ExecutorUtils;
 import org.apache.flink.util.TestLogger;
 import org.apache.flink.util.function.CheckedSupplier;
@@ -32,6 +33,9 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.net.BindException;
+import java.net.InetAddress;
+import java.net.ServerSocket;
 import java.util.HashMap;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
@@ -42,9 +46,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for {@link BootstrapToolsTest}.
@@ -360,4 +367,23 @@ public class BootstrapToolsTest extends TestLogger {
 			ExecutorUtils.gracefulShutdown(10000L, TimeUnit.MILLISECONDS, executorService);
 		}
 	}
+
+	/**
+	 * Tests that the {@link ActorSystem} fails with an expressive exception if it cannot be
+	 * instantiated due to an occupied port.
+	 */
+	@Test
+	public void testActorSystemInstantiationFailureWhenPortOccupied() throws Exception {
+		final ServerSocket portOccupier = new ServerSocket(0, 10, InetAddress.getByName("0.0.0.0"));
+
+		try {
+			final int port = portOccupier.getLocalPort();
+			BootstrapTools.startActorSystem(new Configuration(), "0.0.0.0", port, LOG);
+			fail("Expected to fail with a BindException");
+		} catch (Exception e) {
+			assertThat(ExceptionUtils.findThrowable(e, BindException.class).isPresent(), is(true));
+		} finally {
+			portOccupier.close();
+		}
+	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
deleted file mode 100644
index e535415..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * 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.jobmanager;
-
-import static org.junit.Assert.*;
-import static org.junit.Assume.assumeTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.BindException;
-import java.net.InetAddress;
-import java.net.ServerSocket;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.configuration.BlobServerOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.util.StartupUtils;
-import org.apache.flink.util.NetUtils;
-import org.apache.flink.util.OperatingSystem;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Tests that verify the startup behavior of the JobManager in failure
- * situations, when the JobManager cannot be started.
- */
-public class JobManagerStartupTest extends TestLogger {
-
-	private final static String DOES_NOT_EXISTS_NO_SIR = "does-not-exist-no-sir";
-
-	private File blobStorageDirectory;
-
-	@Rule
-	public final TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws IOException {
-		assumeTrue(!OperatingSystem.isWindows()); //setWritable doesn't work on Windows.
-
-		// Prepare test directory
-		blobStorageDirectory = temporaryFolder.newFolder();
-
-		assertTrue(blobStorageDirectory.setExecutable(true, false));
-		assertTrue(blobStorageDirectory.setReadable(true, false));
-		assertTrue(blobStorageDirectory.setWritable(false, false));
-	}
-
-	@After
-	public void after() {
-		// Cleanup test directory
-		if (blobStorageDirectory != null) {
-			assertTrue(blobStorageDirectory.delete());
-		}
-	}
-
-	/**
-	 * Verifies that the JobManager fails fast (and with expressive error message)
-	 * when the port to listen is already in use.
-	 * @throws Throwable 
-	 */
-	@Test( expected = BindException.class )
-	public void testStartupWithPortInUse() throws BindException {
-		
-		ServerSocket portOccupier;
-		final int portNum;
-		
-		try {
-			portNum = NetUtils.getAvailablePort();
-			portOccupier = new ServerSocket(portNum, 10, InetAddress.getByName("0.0.0.0"));
-		}
-		catch (Throwable t) {
-			// could not find free port, or open a connection there
-			return;
-		}
-		
-		try {
-			JobManager.runJobManager(new Configuration(), JobManagerMode.CLUSTER, "localhost", portNum);
-			fail("this should throw an exception");
-		}
-		catch (Exception e) {
-			// expected
-			List<Throwable> causes = StartupUtils.getExceptionCauses(e, new ArrayList<Throwable>());
-			for(Throwable cause:causes) {
-				if(cause instanceof BindException) {
-					throw (BindException) cause;
-				}	
-			}
-			throw e;
-		}
-		finally {
-			try {
-				portOccupier.close();
-			}
-			catch (Throwable t) {
-				// ignore
-			}
-		}
-	}
-	
-	/**
-	 * Verifies that the JobManager fails fast (and with expressive error message)
-	 * when one of its components (here the BLOB server) fails to start properly.
-	 */
-	@Test
-	public void testJobManagerStartupFails() {
-		final int portNum;
-		try {
-			portNum = NetUtils.getAvailablePort();
-		}
-		catch (Throwable t) {
-			// skip test if we cannot find a free port
-			return;
-		}
-		Configuration failConfig = new Configuration();
-		String nonExistDirectory = new File(blobStorageDirectory, DOES_NOT_EXISTS_NO_SIR).getAbsolutePath();
-		failConfig.setString(BlobServerOptions.STORAGE_DIRECTORY, nonExistDirectory);
-
-		try {
-			JobManager.runJobManager(failConfig, JobManagerMode.CLUSTER, "localhost", portNum);
-			fail("this should fail with an exception");
-		}
-		catch (Exception e) {
-			// expected
-		}
-	}
-}