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