You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2018/12/19 16:51:33 UTC

asterixdb git commit: [ASTERIXDB-2490][NET] Support Encrypted RMI Connections

Repository: asterixdb
Updated Branches:
  refs/heads/master 9b4027cc5 -> 95b8f946e


[ASTERIXDB-2490][NET] Support Encrypted RMI Connections

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- Use RMIServer/ClientFactory in metadata node RMI connections
  to support both unencrypted and encrypted sockets.
- Add config getter to network security manager.

Change-Id: I11577b7d26d8002d4182255fee0dd769945ca389
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3093
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mh...@apache.org>
Reviewed-by: Michael Blow <mb...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/95b8f946
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/95b8f946
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/95b8f946

Branch: refs/heads/master
Commit: 95b8f946e7932b5da5d654886dacfe1eeac9571a
Parents: 9b4027c
Author: Murtadha Hubail <mh...@apache.org>
Authored: Tue Dec 18 04:40:16 2018 +0300
Committer: Murtadha Hubail <mh...@apache.org>
Committed: Wed Dec 19 08:50:52 2018 -0800

----------------------------------------------------------------------
 .../asterix/app/nc/NCAppRuntimeContext.java     | 10 ++++-
 .../hyracks/bootstrap/CCApplication.java        |  3 +-
 .../asterix/metadata/RMIClientFactory.java      | 42 +++++++++++++++++++
 .../asterix/metadata/RMIServerFactory.java      | 44 ++++++++++++++++++++
 .../metadata/bootstrap/AsterixStateProxy.java   | 12 +++++-
 .../api/network/INetworkSecurityManager.java    |  7 ++++
 .../ipc/security/NetworkSecurityManager.java    |  8 ++++
 7 files changed, 122 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/95b8f946/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 8924512..d89004b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -66,6 +66,8 @@ import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataNode;
+import org.apache.asterix.metadata.RMIClientFactory;
+import org.apache.asterix.metadata.RMIServerFactory;
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.api.IMetadataNode;
 import org.apache.asterix.metadata.bootstrap.MetadataBootstrap;
@@ -84,6 +86,7 @@ import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
+import org.apache.hyracks.api.network.INetworkSecurityManager;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.ipc.impl.HyracksConnection;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -430,8 +433,13 @@ public class NCAppRuntimeContext implements INcApplicationContext {
     @Override
     public synchronized void exportMetadataNodeStub() throws RemoteException {
         if (metadataNodeStub == null) {
+            final INetworkSecurityManager networkSecurityManager =
+                    ncServiceContext.getControllerService().getNetworkSecurityManager();
+            final RMIServerFactory serverSocketFactory = new RMIServerFactory(networkSecurityManager);
+            final RMIClientFactory clientSocketFactory =
+                    new RMIClientFactory(networkSecurityManager.getConfiguration().isSslEnabled());
             metadataNodeStub = (IMetadataNode) UnicastRemoteObject.exportObject(MetadataNode.INSTANCE,
-                    getMetadataProperties().getMetadataPort());
+                    getMetadataProperties().getMetadataPort(), clientSocketFactory, serverSocketFactory);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/95b8f946/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 99500ce..24a1463 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -163,7 +163,8 @@ public class CCApplication extends BaseCCApplication {
         }
         MetadataProperties metadataProperties = appCtx.getMetadataProperties();
 
-        setAsterixStateProxy(AsterixStateProxy.registerRemoteObject(metadataProperties.getMetadataCallbackPort()));
+        setAsterixStateProxy(AsterixStateProxy.registerRemoteObject(controllerService.getNetworkSecurityManager(),
+                metadataProperties.getMetadataCallbackPort()));
         ccServiceCtx.setDistributedState(proxy);
         MetadataManager.initialize(proxy, metadataProperties, appCtx);
         ccServiceCtx.addJobLifecycleListener(appCtx.getActiveNotificationHandler());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/95b8f946/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java
new file mode 100644
index 0000000..cba3300
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java
@@ -0,0 +1,42 @@
+
+/*
+ * 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.asterix.metadata;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.Socket;
+import java.rmi.server.RMIClientSocketFactory;
+
+import javax.net.SocketFactory;
+import javax.net.ssl.SSLSocketFactory;
+
+public class RMIClientFactory implements RMIClientSocketFactory, Serializable {
+
+    private final boolean sslEnabled;
+
+    public RMIClientFactory(boolean sslEnabled) {
+        this.sslEnabled = sslEnabled;
+    }
+
+    public Socket createSocket(String host, int port) throws IOException {
+        final SocketFactory factory = sslEnabled ? SSLSocketFactory.getDefault() : SocketFactory.getDefault();
+        return factory.createSocket(host, port);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/95b8f946/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java
new file mode 100644
index 0000000..9506c5a
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIServerFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.asterix.metadata;
+
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.rmi.server.RMIServerSocketFactory;
+
+import javax.net.ServerSocketFactory;
+
+import org.apache.hyracks.api.network.INetworkSecurityManager;
+
+public class RMIServerFactory implements RMIServerSocketFactory {
+
+    private final INetworkSecurityManager securityManager;
+
+    public RMIServerFactory(INetworkSecurityManager securityManager) {
+        this.securityManager = securityManager;
+    }
+
+    @Override
+    public ServerSocket createServerSocket(int port) throws IOException {
+        if (securityManager.getConfiguration().isSslEnabled()) {
+            return securityManager.newSSLContext().getServerSocketFactory().createServerSocket(port);
+        }
+        return ServerSocketFactory.getDefault().createServerSocket(port);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/95b8f946/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
index 5357fc8..4c971e2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
@@ -23,8 +23,11 @@ import java.rmi.RemoteException;
 import java.rmi.server.UnicastRemoteObject;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.asterix.metadata.RMIClientFactory;
+import org.apache.asterix.metadata.RMIServerFactory;
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.api.IMetadataNode;
+import org.apache.hyracks.api.network.INetworkSecurityManager;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -38,8 +41,13 @@ public class AsterixStateProxy implements IAsterixStateProxy {
     private IMetadataNode metadataNode;
     private static final IAsterixStateProxy cc = new AsterixStateProxy();
 
-    public static IAsterixStateProxy registerRemoteObject(int metadataCallbackPort) throws RemoteException {
-        IAsterixStateProxy stub = (IAsterixStateProxy) UnicastRemoteObject.exportObject(cc, metadataCallbackPort);
+    public static IAsterixStateProxy registerRemoteObject(INetworkSecurityManager networkSecurityManager,
+            int metadataCallbackPort) throws RemoteException {
+        final RMIServerFactory serverSocketFactory = new RMIServerFactory(networkSecurityManager);
+        final RMIClientFactory clientSocketFactory =
+                new RMIClientFactory(networkSecurityManager.getConfiguration().isSslEnabled());
+        final IAsterixStateProxy stub = (IAsterixStateProxy) UnicastRemoteObject.exportObject(cc, metadataCallbackPort,
+                clientSocketFactory, serverSocketFactory);
         LOGGER.info("Asterix Distributed State Proxy Bound");
         return stub;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/95b8f946/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityManager.java
index 9dc6960..2cdf525 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityManager.java
@@ -50,4 +50,11 @@ public interface INetworkSecurityManager {
      * @return the socket channel factory
      */
     ISocketChannelFactory getSocketChannelFactory();
+
+    /**
+     * Gets the current configuration of this {@link INetworkSecurityManager}
+     *
+     * @return the current configuration
+     */
+    INetworkSecurityConfig getConfiguration();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/95b8f946/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
index ed25f41..310eee5 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
@@ -41,6 +41,9 @@ public class NetworkSecurityManager implements INetworkSecurityManager {
 
     public NetworkSecurityManager(INetworkSecurityConfig config) {
         this.config = config;
+        if (config.isSslEnabled()) {
+            System.setProperty("javax.net.ssl.trustStore", config.getTrustStoreFile().getAbsolutePath());
+        }
         sslSocketFactory = new SslSocketChannelFactory(this);
     }
 
@@ -84,6 +87,11 @@ public class NetworkSecurityManager implements INetworkSecurityManager {
     }
 
     @Override
+    public INetworkSecurityConfig getConfiguration() {
+        return config;
+    }
+
+    @Override
     public void setConfiguration(INetworkSecurityConfig config) {
         this.config = config;
     }