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");
+ }
+ }
+}