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 2018/07/16 07:03:11 UTC

[6/8] flink git commit: [FLINK-9313] [security] (part 3) Activate mutual authentication for RPC/akka

[FLINK-9313] [security] (part 3) Activate mutual authentication for RPC/akka


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3aeb00f6
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3aeb00f6
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3aeb00f6

Branch: refs/heads/master
Commit: 3aeb00f6cf103407245fc192ce5940ee4217d6b1
Parents: 30c4bc8
Author: Stephan Ewen <se...@apache.org>
Authored: Mon May 7 19:44:33 2018 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Mon Jul 16 08:10:46 2018 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/akka/AkkaUtils.scala   |   1 +
 .../flink/runtime/rpc/RpcSSLAuthITCase.java     | 162 +++++++++++++++++++
 2 files changed, 163 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3aeb00f6/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
index 12378e0..b58bfe1 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
@@ -453,6 +453,7 @@ object AkkaUtils {
          |          protocol = $akkaSSLProtocol
          |          enabled-algorithms = $akkaSSLAlgorithms
          |          random-number-generator = ""
+         |          require-mutual-authentication = on
          |        }
          |      }
          |    }

http://git-wip-us.apache.org/repos/asf/flink/blob/3aeb00f6/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java
new file mode 100644
index 0000000..a0e239e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java
@@ -0,0 +1,162 @@
+/*
+ * 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.rpc;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.SecurityOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
+import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import akka.actor.Terminated;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This test validates that the RPC service gives a good message when it cannot
+ * connect to an RpcEndpoint.
+ */
+public class RpcSSLAuthITCase extends TestLogger {
+
+	private static final String KEY_STORE_FILE = RpcSSLAuthITCase.class.getResource("/local127.keystore").getFile();
+	private static final String TRUST_STORE_FILE = RpcSSLAuthITCase.class.getResource("/local127.truststore").getFile();
+	private static final String UNTRUSTED_KEY_STORE_FILE = RpcSSLAuthITCase.class.getResource("/untrusted.keystore").getFile();
+
+	@Test
+	public void testConnectFailure() throws Exception {
+
+		// !!! This config has KEY_STORE_FILE / TRUST_STORE_FILE !!!
+		Configuration sslConfig1 = new Configuration();
+		sslConfig1.setBoolean(SecurityOptions.SSL_INTERNAL_ENABLED, true);
+		sslConfig1.setString(SecurityOptions.SSL_INTERNAL_KEYSTORE, KEY_STORE_FILE);
+		sslConfig1.setString(SecurityOptions.SSL_INTERNAL_TRUSTSTORE, TRUST_STORE_FILE);
+		sslConfig1.setString(SecurityOptions.SSL_INTERNAL_KEYSTORE_PASSWORD, "password");
+		sslConfig1.setString(SecurityOptions.SSL_INTERNAL_KEY_PASSWORD, "password");
+		sslConfig1.setString(SecurityOptions.SSL_INTERNAL_TRUSTSTORE_PASSWORD, "password");
+		sslConfig1.setString(SecurityOptions.SSL_ALGORITHMS, "TLS_RSA_WITH_AES_128_CBC_SHA");
+
+		// !!! This config has KEY_STORE_FILE / UNTRUSTED_KEY_STORE_FILE !!!
+		// If this is presented by a client, it will trust the server, but the server will
+		// not trust this client in case client auth is enabled.
+		Configuration sslConfig2 = new Configuration();
+		sslConfig2.setBoolean(SecurityOptions.SSL_INTERNAL_ENABLED, true);
+		sslConfig2.setString(SecurityOptions.SSL_INTERNAL_KEYSTORE, UNTRUSTED_KEY_STORE_FILE);
+		sslConfig2.setString(SecurityOptions.SSL_INTERNAL_TRUSTSTORE, TRUST_STORE_FILE);
+		sslConfig2.setString(SecurityOptions.SSL_INTERNAL_KEYSTORE_PASSWORD, "password");
+		sslConfig2.setString(SecurityOptions.SSL_INTERNAL_KEY_PASSWORD, "password");
+		sslConfig2.setString(SecurityOptions.SSL_INTERNAL_TRUSTSTORE_PASSWORD, "password");
+		sslConfig2.setString(SecurityOptions.SSL_ALGORITHMS, "TLS_RSA_WITH_AES_128_CBC_SHA");
+
+		ActorSystem actorSystem1 = null;
+		ActorSystem actorSystem2 = null;
+		RpcService rpcService1 = null;
+		RpcService rpcService2 = null;
+
+		try {
+			actorSystem1 = AkkaUtils.createActorSystem(sslConfig1, "localhost", 0);
+			actorSystem2 = AkkaUtils.createActorSystem(sslConfig2, "localhost", 0);
+
+			// to test whether the test is still good:
+			//   - create actorSystem2 with sslConfig1 (same as actorSystem1) and see that both can connect
+			//   - set 'require-mutual-authentication = off' in the AkkaUtils ssl config section
+
+			// we start the RPC service with a very long timeout to ensure that the test
+			// can only pass if the connection problem is not recognized merely via a timeout
+			rpcService1 = new AkkaRpcService(actorSystem1, Time.of(10000000, TimeUnit.SECONDS));
+			rpcService2 = new AkkaRpcService(actorSystem2, Time.of(10000000, TimeUnit.SECONDS));
+
+			TestEndpoint endpoint = new TestEndpoint(rpcService1);
+			endpoint.start();
+
+			CompletableFuture<TestGateway> future = rpcService2.connect(endpoint.getAddress(), TestGateway.class);
+			TestGateway gateway = future.get(10000000, TimeUnit.SECONDS);
+
+			CompletableFuture<String> fooFuture = gateway.foo();
+			fooFuture.get();
+
+			fail("should never complete normally");
+		}
+		catch (ExecutionException e) {
+			// that is what we want
+			assertTrue(e.getCause() instanceof RpcConnectionException);
+		}
+		finally {
+			final CompletableFuture<Void> rpcTerminationFuture1 = rpcService1 != null ?
+					rpcService1.stopService() :
+					CompletableFuture.completedFuture(null);
+
+			final CompletableFuture<Void> rpcTerminationFuture2 = rpcService2 != null ?
+					rpcService2.stopService() :
+					CompletableFuture.completedFuture(null);
+
+			final CompletableFuture<Terminated> actorSystemTerminationFuture1 = actorSystem1 != null ?
+					FutureUtils.toJava(actorSystem1.terminate()) :
+					CompletableFuture.completedFuture(null);
+
+			final CompletableFuture<Terminated> actorSystemTerminationFuture2 = actorSystem2 != null ?
+					FutureUtils.toJava(actorSystem2.terminate()) :
+					CompletableFuture.completedFuture(null);
+
+			FutureUtils
+				.waitForAll(Arrays.asList(
+						rpcTerminationFuture1, rpcTerminationFuture2,
+						actorSystemTerminationFuture1, actorSystemTerminationFuture2))
+				.get();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Test RPC endpoint
+	// ------------------------------------------------------------------------
+
+	/** doc. */
+	public interface TestGateway extends RpcGateway {
+
+		CompletableFuture<String> foo();
+	}
+
+	/** doc. */
+	public static class TestEndpoint extends RpcEndpoint implements TestGateway {
+
+		public TestEndpoint(RpcService rpcService) {
+			super(rpcService);
+		}
+
+		@Override
+		public CompletableFuture<Void> postStop() {
+			return CompletableFuture.completedFuture(null);
+		}
+
+		@Override
+		public CompletableFuture<String> foo() {
+			return CompletableFuture.completedFuture("bar");
+		}
+	}
+}