You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2016/03/07 21:29:36 UTC

[1/7] storm git commit: Adding Plain Sasl Transport Plugin

Repository: storm
Updated Branches:
  refs/heads/master 4117fe545 -> 974274372


Adding Plain Sasl Transport Plugin


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

Branch: refs/heads/master
Commit: b1e4c94269dbcf2ba01aee89e468c744887c53de
Parents: d42c437
Author: Kishor Patil <kp...@yahoo-inc.com>
Authored: Mon Feb 29 14:44:52 2016 -0600
Committer: Kishor Patil <kp...@yahoo-inc.com>
Committed: Tue Mar 1 16:18:04 2016 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |   2 +-
 .../auth/plain/PlainClientCallbackHandler.java  |  80 ++++++++++
 .../auth/plain/PlainSaslTransportPlugin.java    |  80 ++++++++++
 .../auth/plain/PlainServerCallbackHandler.java  | 108 +++++++++++++
 .../security/auth/plain/SaslPlainServer.java    | 154 +++++++++++++++++++
 5 files changed, 423 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b1e4c942/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 9817161..b32c2ff 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -39,7 +39,7 @@ storm.exhibitor.port: 8080
 storm.exhibitor.poll.uripath: "/exhibitor/v1/cluster/list"
 storm.cluster.mode: "distributed" # can be distributed or local
 storm.local.mode.zmq: false
-storm.thrift.transport: "org.apache.storm.security.auth.SimpleTransportPlugin"
+storm.thrift.transport: "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin"
 storm.principal.tolocal: "org.apache.storm.security.auth.DefaultPrincipalToLocal"
 storm.group.mapping.service: "org.apache.storm.security.auth.ShellBasedGroupsMapping"
 storm.group.mapping.service.params: null

http://git-wip-us.apache.org/repos/asf/storm/blob/b1e4c942/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
new file mode 100644
index 0000000..25c7609
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
@@ -0,0 +1,80 @@
+/**
+ * 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.storm.security.auth.plain;
+
+import java.io.IOException;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  client side callback handler.
+ */
+public class PlainClientCallbackHandler implements CallbackHandler {
+    private static final String USERNAME = "username";
+    private static final String PASSWORD = "password";
+    private static final Logger LOG = LoggerFactory.getLogger(PlainClientCallbackHandler.class);
+    private String _username = "username";
+    private String _password = "password";
+
+    /**
+     * This method is invoked by SASL for authentication challenges
+     * @param callbacks a collection of challenge callbacks 
+     */
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        for (Callback c : callbacks) {
+            if (c instanceof NameCallback) {
+                LOG.debug("name callback");
+                NameCallback nc = (NameCallback) c;
+                nc.setName(_username);
+            } else if (c instanceof PasswordCallback) {
+                LOG.debug("password callback");
+                PasswordCallback pc = (PasswordCallback)c;
+                if (_password != null) {
+                    pc.setPassword(_password.toCharArray());
+                } 
+            } else if (c instanceof AuthorizeCallback) {
+                LOG.debug("authorization callback");
+                AuthorizeCallback ac = (AuthorizeCallback) c;
+                String authid = ac.getAuthenticationID();
+                String authzid = ac.getAuthorizationID();
+                if (authid.equals(authzid)) {
+                    ac.setAuthorized(true);
+                } else {
+                    ac.setAuthorized(false);
+                }
+                if (ac.isAuthorized()) {
+                    ac.setAuthorizedID(authzid);
+                }
+            } else if (c instanceof RealmCallback) {
+                RealmCallback rc = (RealmCallback) c;
+                ((RealmCallback) c).setText(rc.getDefaultText());
+            } else {
+                throw new UnsupportedCallbackException(c);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b1e4c942/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
new file mode 100644
index 0000000..facc352
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
@@ -0,0 +1,80 @@
+/**
+ * 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.storm.security.auth.plain;
+
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.SaslTransportPlugin;
+import org.apache.storm.utils.ExtendedThreadPoolExecutor;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.server.TThreadPoolServer;
+import org.apache.thrift.transport.TSaslClientTransport;
+import org.apache.thrift.transport.TSaslServerTransport;
+import org.apache.thrift.transport.TServerSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.thrift.transport.TTransportFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.callback.CallbackHandler;
+import java.io.IOException;
+import java.security.Security;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public class PlainSaslTransportPlugin extends SaslTransportPlugin {
+    public static final String PLAIN = "PLAIN";
+    private static final Logger LOG = LoggerFactory.getLogger(PlainSaslTransportPlugin.class);
+
+    @Override
+    protected TTransportFactory getServerTransportFactory() throws IOException {
+        //create an authentication callback handler
+        CallbackHandler serer_callback_handler = new PlainServerCallbackHandler();
+        Security.addProvider(new SaslPlainServer.SecurityProvider());
+        //create a transport factory that will invoke our auth callback for digest
+        TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory();
+        factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, serer_callback_handler);
+
+        LOG.info("SASL PLAIN transport factory will be used");
+        return factory;
+    }
+
+    @Override
+    public TTransport connect(TTransport transport, String serverHost, String asUser) throws IOException, TTransportException {
+        PlainClientCallbackHandler client_callback_handler = new PlainClientCallbackHandler();
+        TSaslClientTransport wrapper_transport = new TSaslClientTransport(PLAIN,
+            null,
+            AuthUtils.SERVICE,
+            serverHost,
+            null,
+            client_callback_handler,
+            transport);
+
+        wrapper_transport.open();
+        LOG.debug("SASL PLAIN client transport has been established");
+
+        return wrapper_transport;
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b1e4c942/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
new file mode 100644
index 0000000..e1ae2d9
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
@@ -0,0 +1,108 @@
+/**
+ * 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.storm.security.auth.plain;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.security.auth.SaslTransportPlugin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+
+/**
+ * SASL server side callback handler
+ */
+public class PlainServerCallbackHandler implements CallbackHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class);
+    private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword";
+
+    private String userName="username";
+    private final Map<String,String> credentials = new HashMap<>();
+
+    public PlainServerCallbackHandler() throws IOException {
+        credentials.put("username", "password");
+    }
+
+    public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+        for (Callback callback : callbacks) {
+            if (callback instanceof NameCallback) {
+                handleNameCallback((NameCallback) callback);
+            } else if (callback instanceof PasswordCallback) {
+                handlePasswordCallback((PasswordCallback) callback);
+            } else if (callback instanceof RealmCallback) {
+                handleRealmCallback((RealmCallback) callback);
+            } else if (callback instanceof AuthorizeCallback) {
+                handleAuthorizeCallback((AuthorizeCallback) callback);
+            }
+        }
+    }
+
+    private void handleNameCallback(NameCallback nc) {
+        LOG.debug("handleNameCallback");
+        userName = nc.getDefaultName();
+        nc.setName(nc.getDefaultName());
+    }
+
+    private void handlePasswordCallback(PasswordCallback pc) {
+        LOG.debug("handlePasswordCallback");
+        if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) {
+            // superuser: use Java system property for password, if available.
+            pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray());
+        } else if (credentials.containsKey(userName) ) {
+            pc.setPassword(credentials.get(userName).toCharArray());
+        } else {
+            LOG.warn("No password found for user: " + userName);
+        }
+    }
+
+    private void handleRealmCallback(RealmCallback rc) {
+        LOG.debug("handleRealmCallback: "+ rc.getDefaultText());
+        rc.setText(rc.getDefaultText());
+    }
+
+    private void handleAuthorizeCallback(AuthorizeCallback ac) {
+        String authenticationID = ac.getAuthenticationID();
+        LOG.info("Successfully authenticated client: authenticationID = " + authenticationID + " authorizationID = " + ac.getAuthorizationID());
+
+        //if authorizationId is not set, set it to authenticationId.
+        if(ac.getAuthorizationID() == null) {
+            ac.setAuthorizedID(authenticationID);
+        }
+
+        //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We
+        //add the authNid as the real user in reqContext's subject which will be used during authorization.
+        if(!authenticationID.equals(ac.getAuthorizationID())) {
+            LOG.info("Impersonation attempt  authenticationID = " + ac.getAuthenticationID() + " authorizationID = " + ac.getAuthorizationID());
+            ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID()));
+        } else {
+            ReqContext.context().setRealPrincipal(null);
+        }
+
+        ac.setAuthorized(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b1e4c942/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
new file mode 100644
index 0000000..a76c481
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
@@ -0,0 +1,154 @@
+/**
+ * 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.storm.security.auth.plain;
+
+import java.security.Provider;
+import java.util.Map;
+
+import javax.security.auth.callback.*;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+
+public class SaslPlainServer implements SaslServer {
+  @SuppressWarnings("serial")
+  public static class SecurityProvider extends Provider {
+    public SecurityProvider() {
+      super("SaslPlainServer", 1.0, "SASL PLAIN Authentication Server");
+      put("SaslServerFactory.PLAIN",
+          SaslPlainServerFactory.class.getName());
+    }
+  }
+
+  public static class SaslPlainServerFactory implements SaslServerFactory {
+    @Override
+    public SaslServer createSaslServer(String mechanism, String protocol,
+        String serverName, Map<String,?> props, CallbackHandler cbh)
+            throws SaslException {
+      return "PLAIN".equals(mechanism) ? new SaslPlainServer(cbh) : null; 
+    }
+    @Override
+    public String[] getMechanismNames(Map<String,?> props){
+      return (props == null) || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT))
+          ? new String[]{"PLAIN"}
+          : new String[0];
+    }
+  }
+  
+  private CallbackHandler cbh;
+  private boolean completed;
+  private String authz;
+  
+  SaslPlainServer(CallbackHandler callback) {
+    this.cbh = callback;
+  }
+
+  @Override
+  public String getMechanismName() {
+    return "PLAIN";
+  }
+  
+  @Override
+  public byte[] evaluateResponse(byte[] response) throws SaslException {
+    if (completed) {
+      throw new IllegalStateException("PLAIN authentication has completed");
+    }
+    if (response == null) {
+      throw new IllegalArgumentException("Received null response");
+    }
+    try {
+      String payload;
+      try {
+        payload = new String(response, "UTF-8");
+      } catch (Exception e) {
+        throw new IllegalArgumentException("Received corrupt response", e);
+      }
+      // [ authz, authn, password ]
+      String[] parts = payload.split("\u0000", 3);
+      if (parts.length != 3) {
+        throw new IllegalArgumentException("Received corrupt response");
+      }
+      if (parts[0].isEmpty()) { // authz = authn
+        parts[0] = parts[1];
+      }
+      
+      NameCallback nc = new NameCallback("SASL PLAIN");
+      nc.setName(parts[1]);
+      PasswordCallback pc = new PasswordCallback("SASL PLAIN", false);
+      pc.setPassword(parts[2].toCharArray());
+      AuthorizeCallback ac = new AuthorizeCallback(parts[1], parts[0]);
+      cbh.handle(new Callback[]{nc, pc, ac});      
+      if (ac.isAuthorized()) {
+        authz = ac.getAuthorizedID();
+      }
+    } catch (Exception e) {
+      throw new SaslException("PLAIN auth failed: " + e.toString(), e);
+    } finally {
+      completed = true;
+    }
+    return null;
+  }
+
+  private void throwIfNotComplete() {
+    if (!completed) {
+      throw new IllegalStateException("PLAIN authentication not completed");
+    }
+  }
+  
+  @Override
+  public boolean isComplete() {
+    return completed;
+  }
+
+  @Override
+  public String getAuthorizationID() {
+    throwIfNotComplete();
+    return authz;
+  }
+  
+  @Override
+  public Object getNegotiatedProperty(String propName) {
+    throwIfNotComplete();      
+    return Sasl.QOP.equals(propName) ? "auth" : null;
+  }
+  
+  @Override
+  public byte[] wrap(byte[] outgoing, int offset, int len)
+      throws SaslException {
+    throwIfNotComplete();
+    throw new IllegalStateException(
+        "PLAIN supports neither integrity nor privacy");      
+  }
+  
+  @Override
+  public byte[] unwrap(byte[] incoming, int offset, int len)
+      throws SaslException {
+    throwIfNotComplete();
+    throw new IllegalStateException(
+        "PLAIN supports neither integrity nor privacy");      
+  }
+  
+  @Override
+  public void dispose() throws SaslException {
+    cbh = null;
+    authz = null;
+  }
+}


[3/7] storm git commit: Addressing review comments

Posted by bo...@apache.org.
Addressing review comments


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/4e04ce8d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/4e04ce8d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/4e04ce8d

Branch: refs/heads/master
Commit: 4e04ce8dcdc33d488a3d15f7a47ab8af15136db4
Parents: cccb976
Author: Kishor Patil <kp...@yahoo-inc.com>
Authored: Wed Mar 2 10:27:56 2016 -0600
Committer: Kishor Patil <kp...@yahoo-inc.com>
Committed: Wed Mar 2 10:27:56 2016 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |  2 +-
 .../auth/AbstractSaslServerCallbackHandler.java | 21 ++++++++++++++++++--
 .../auth/plain/PlainSaslTransportPlugin.java    | 14 +++++++------
 .../security/auth/plain/SaslPlainServer.java    |  5 ++++-
 4 files changed, 32 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4e04ce8d/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index b32c2ff..9817161 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -39,7 +39,7 @@ storm.exhibitor.port: 8080
 storm.exhibitor.poll.uripath: "/exhibitor/v1/cluster/list"
 storm.cluster.mode: "distributed" # can be distributed or local
 storm.local.mode.zmq: false
-storm.thrift.transport: "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin"
+storm.thrift.transport: "org.apache.storm.security.auth.SimpleTransportPlugin"
 storm.principal.tolocal: "org.apache.storm.security.auth.DefaultPrincipalToLocal"
 storm.group.mapping.service: "org.apache.storm.security.auth.ShellBasedGroupsMapping"
 storm.group.mapping.service.params: null

http://git-wip-us.apache.org/repos/asf/storm/blob/4e04ce8d/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java
index 0a57f93..ebbe2ea 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java
@@ -1,3 +1,20 @@
+/**
+ * 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.storm.security.auth;
 
 import org.slf4j.Logger;
@@ -43,12 +60,12 @@ public abstract class AbstractSaslServerCallbackHandler implements CallbackHandl
         if (credentials.containsKey(userName) ) {
             pc.setPassword(credentials.get(userName).toCharArray());
         } else {
-            LOG.warn("No password found for user: " + userName);
+            LOG.warn("No password found for user: {}", userName);
         }
     }
 
     private void handleRealmCallback(RealmCallback rc) {
-        LOG.debug("handleRealmCallback: "+ rc.getDefaultText());
+        LOG.debug("handleRealmCallback: {}", rc.getDefaultText());
         rc.setText(rc.getDefaultText());
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4e04ce8d/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
index 211a4b7..6247fe6 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
@@ -39,7 +39,9 @@ public class PlainSaslTransportPlugin extends SaslTransportPlugin {
     protected TTransportFactory getServerTransportFactory() throws IOException {
         //create an authentication callback handler
         CallbackHandler server_callback_handler = new PlainServerCallbackHandler();
-        Security.addProvider(new SaslPlainServer.SecurityProvider());
+        if (Security.getProvider(SaslPlainServer.SecurityProvider.SASL_PLAIN_SERVER) == null) {
+            Security.addProvider(new SaslPlainServer.SecurityProvider());
+        }
         //create a transport factory that will invoke our auth callback for digest
         TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory();
         factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, server_callback_handler);
@@ -50,19 +52,19 @@ public class PlainSaslTransportPlugin extends SaslTransportPlugin {
 
     @Override
     public TTransport connect(TTransport transport, String serverHost, String asUser) throws IOException, TTransportException {
-        PlainClientCallbackHandler client_callback_handler = new PlainClientCallbackHandler();
-        TSaslClientTransport wrapper_transport = new TSaslClientTransport(PLAIN,
+        PlainClientCallbackHandler clientCallbackHandler = new PlainClientCallbackHandler();
+        TSaslClientTransport wrapperTransport = new TSaslClientTransport(PLAIN,
             null,
             AuthUtils.SERVICE,
             serverHost,
             null,
-            client_callback_handler,
+            clientCallbackHandler,
             transport);
 
-        wrapper_transport.open();
+        wrapperTransport.open();
         LOG.debug("SASL PLAIN client transport has been established");
 
-        return wrapper_transport;
+        return wrapperTransport;
 
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4e04ce8d/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
index dd2582c..c84ce77 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
@@ -32,8 +32,11 @@ import java.util.Map;
 public class SaslPlainServer implements SaslServer {
   @SuppressWarnings("serial")
   public static class SecurityProvider extends Provider {
+
+    public static final String SASL_PLAIN_SERVER = "SaslPlainServer";
+
     public SecurityProvider() {
-      super("SaslPlainServer", 1.0, "SASL PLAIN Authentication Server");
+      super(SASL_PLAIN_SERVER, 1.0, "SASL PLAIN Authentication Server");
       put("SaslServerFactory.PLAIN",
           SaslPlainServerFactory.class.getName());
     }


[2/7] storm git commit: Refactoring SaslServerCallbackHandler and SaslClientCallbackHandler

Posted by bo...@apache.org.
Refactoring SaslServerCallbackHandler and SaslClientCallbackHandler


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

Branch: refs/heads/master
Commit: cccb9766eb6b01477b44cd35e836997811464632
Parents: b1e4c94
Author: Kishor Patil <kp...@yahoo-inc.com>
Authored: Wed Mar 2 00:12:57 2016 -0600
Committer: Kishor Patil <kp...@yahoo-inc.com>
Committed: Wed Mar 2 00:12:57 2016 -0600

----------------------------------------------------------------------
 .../auth/AbstractSaslClientCallbackHandler.java | 76 +++++++++++++++++++
 .../auth/AbstractSaslServerCallbackHandler.java | 77 ++++++++++++++++++++
 .../auth/digest/ClientCallbackHandler.java      | 60 ++-------------
 .../auth/digest/ServerCallbackHandler.java      | 61 ++--------------
 .../auth/plain/PlainClientCallbackHandler.java  | 63 ++--------------
 .../auth/plain/PlainSaslTransportPlugin.java    | 15 +---
 .../auth/plain/PlainServerCallbackHandler.java  | 66 +----------------
 .../security/auth/plain/SaslPlainServer.java    | 13 ++--
 8 files changed, 184 insertions(+), 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java
new file mode 100644
index 0000000..04710ba
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.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.storm.security.auth;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+import java.io.IOException;
+
+public abstract class AbstractSaslClientCallbackHandler implements CallbackHandler {
+    protected static final String USERNAME = "username";
+    protected static final String PASSWORD = "password";
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractSaslClientCallbackHandler.class);
+    protected String _username = null;
+    protected String _password = null;
+
+    /**
+     * This method is invoked by SASL for authentication challenges
+     * @param callbacks a collection of challenge callbacks
+     */
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        for (Callback c : callbacks) {
+            if (c instanceof NameCallback) {
+                LOG.debug("name callback");
+                NameCallback nc = (NameCallback) c;
+                nc.setName(_username);
+            } else if (c instanceof PasswordCallback) {
+                LOG.debug("password callback");
+                PasswordCallback pc = (PasswordCallback)c;
+                if (_password != null) {
+                    pc.setPassword(_password.toCharArray());
+                }
+            } else if (c instanceof AuthorizeCallback) {
+                LOG.debug("authorization callback");
+                AuthorizeCallback ac = (AuthorizeCallback) c;
+                String authid = ac.getAuthenticationID();
+                String authzid = ac.getAuthorizationID();
+                if (authid.equals(authzid)) {
+                    ac.setAuthorized(true);
+                } else {
+                    ac.setAuthorized(false);
+                }
+                if (ac.isAuthorized()) {
+                    ac.setAuthorizedID(authzid);
+                }
+            } else if (c instanceof RealmCallback) {
+                RealmCallback rc = (RealmCallback) c;
+                ((RealmCallback) c).setText(rc.getDefaultText());
+            } else {
+                throw new UnsupportedCallbackException(c);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java
new file mode 100644
index 0000000..0a57f93
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java
@@ -0,0 +1,77 @@
+package org.apache.storm.security.auth;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class AbstractSaslServerCallbackHandler implements CallbackHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractSaslServerCallbackHandler.class);
+    protected final Map<String,String> credentials = new HashMap<>();
+    protected String userName;
+
+    public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+        for (Callback callback : callbacks) {
+            if (callback instanceof NameCallback) {
+                handleNameCallback((NameCallback) callback);
+            } else if (callback instanceof PasswordCallback) {
+                handlePasswordCallback((PasswordCallback) callback);
+            } else if (callback instanceof RealmCallback) {
+                handleRealmCallback((RealmCallback) callback);
+            } else if (callback instanceof AuthorizeCallback) {
+                handleAuthorizeCallback((AuthorizeCallback) callback);
+            }
+        }
+    }
+
+    private void handleNameCallback(NameCallback nc) {
+        LOG.debug("handleNameCallback");
+        userName = nc.getDefaultName();
+        nc.setName(nc.getDefaultName());
+    }
+
+    protected void handlePasswordCallback(PasswordCallback pc) {
+        LOG.debug("handlePasswordCallback");
+        if (credentials.containsKey(userName) ) {
+            pc.setPassword(credentials.get(userName).toCharArray());
+        } else {
+            LOG.warn("No password found for user: " + userName);
+        }
+    }
+
+    private void handleRealmCallback(RealmCallback rc) {
+        LOG.debug("handleRealmCallback: "+ rc.getDefaultText());
+        rc.setText(rc.getDefaultText());
+    }
+
+    private void handleAuthorizeCallback(AuthorizeCallback ac) {
+        String authenticationID = ac.getAuthenticationID();
+        LOG.info("Successfully authenticated client: authenticationID = {} authorizationID = {}",
+            authenticationID, ac.getAuthorizationID());
+
+        //if authorizationId is not set, set it to authenticationId.
+        if(ac.getAuthorizationID() == null) {
+            ac.setAuthorizedID(authenticationID);
+        }
+
+        //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We
+        //add the authNid as the real user in reqContext's subject which will be used during authorization.
+        if(!authenticationID.equals(ac.getAuthorizationID())) {
+            LOG.info("Impersonation attempt  authenticationID = {} authorizationID = {}",
+                ac.getAuthenticationID(),  ac.getAuthorizationID());
+            ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID()));
+        } else {
+            ReqContext.context().setRealPrincipal(null);
+        }
+
+        ac.setAuthorized(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java
index 013ce06..312e4ab 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java
@@ -17,30 +17,17 @@
  */
 package org.apache.storm.security.auth.digest;
 
-import java.io.IOException;
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.CallbackHandler;
-import javax.security.auth.callback.NameCallback;
-import javax.security.auth.callback.PasswordCallback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.RealmCallback;
+import org.apache.storm.security.auth.AbstractSaslClientCallbackHandler;
+import org.apache.storm.security.auth.AuthUtils;
+
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.storm.security.auth.AuthUtils;
+import java.io.IOException;
 
 /**
  *  client side callback handler.
  */
-public class ClientCallbackHandler implements CallbackHandler {
-    private static final String USERNAME = "username";
-    private static final String PASSWORD = "password";
-    private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class);
-    private String _username = null;
-    private String _password = null;
+public class ClientCallbackHandler extends AbstractSaslClientCallbackHandler {
 
     /**
      * Constructor based on a JAAS configuration
@@ -68,41 +55,4 @@ public class ClientCallbackHandler implements CallbackHandler {
         }
     }
 
-    /**
-     * This method is invoked by SASL for authentication challenges
-     * @param callbacks a collection of challenge callbacks 
-     */
-    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
-        for (Callback c : callbacks) {
-            if (c instanceof NameCallback) {
-                LOG.debug("name callback");
-                NameCallback nc = (NameCallback) c;
-                nc.setName(_username);
-            } else if (c instanceof PasswordCallback) {
-                LOG.debug("password callback");
-                PasswordCallback pc = (PasswordCallback)c;
-                if (_password != null) {
-                    pc.setPassword(_password.toCharArray());
-                } 
-            } else if (c instanceof AuthorizeCallback) {
-                LOG.debug("authorization callback");
-                AuthorizeCallback ac = (AuthorizeCallback) c;
-                String authid = ac.getAuthenticationID();
-                String authzid = ac.getAuthorizationID();
-                if (authid.equals(authzid)) {
-                    ac.setAuthorized(true);
-                } else {
-                    ac.setAuthorized(false);
-                }
-                if (ac.isAuthorized()) {
-                    ac.setAuthorizedID(authzid);
-                }
-            } else if (c instanceof RealmCallback) {
-                RealmCallback rc = (RealmCallback) c;
-                ((RealmCallback) c).setText(rc.getDefaultText());
-            } else {
-                throw new UnsupportedCallbackException(c);
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java
index 4fe21c2..7c4414f 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.storm.security.auth.AbstractSaslServerCallbackHandler;
 import org.apache.storm.security.auth.ReqContext;
 import org.apache.storm.security.auth.SaslTransportPlugin;
 import org.slf4j.Logger;
@@ -41,13 +42,10 @@ import org.apache.storm.security.auth.AuthUtils;
 /**
  * SASL server side callback handler
  */
-public class ServerCallbackHandler implements CallbackHandler {
-    private static final String USER_PREFIX = "user_";
+public class ServerCallbackHandler extends AbstractSaslServerCallbackHandler {
     private static final Logger LOG = LoggerFactory.getLogger(ServerCallbackHandler.class);
-    private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword";
-
-    private String userName;
-    private final Map<String,String> credentials = new HashMap<>();
+    private static final String USER_PREFIX = "user_";
+    public static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword";
 
     public ServerCallbackHandler(Configuration configuration) throws IOException {
         if (configuration==null) return;
@@ -72,61 +70,16 @@ public class ServerCallbackHandler implements CallbackHandler {
         }
     }
 
-    public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
-        for (Callback callback : callbacks) {
-            if (callback instanceof NameCallback) {
-                handleNameCallback((NameCallback) callback);
-            } else if (callback instanceof PasswordCallback) {
-                handlePasswordCallback((PasswordCallback) callback);
-            } else if (callback instanceof RealmCallback) {
-                handleRealmCallback((RealmCallback) callback);
-            } else if (callback instanceof AuthorizeCallback) {
-                handleAuthorizeCallback((AuthorizeCallback) callback);
-            }
-        }
-    }
-
-    private void handleNameCallback(NameCallback nc) {
-        LOG.debug("handleNameCallback");
-        userName = nc.getDefaultName();
-        nc.setName(nc.getDefaultName());
-    }
-
-    private void handlePasswordCallback(PasswordCallback pc) {
+    @Override
+    protected void handlePasswordCallback(PasswordCallback pc) {
         LOG.debug("handlePasswordCallback");
         if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) {
             // superuser: use Java system property for password, if available.
             pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray());
-        } else if (credentials.containsKey(userName) ) {
-            pc.setPassword(credentials.get(userName).toCharArray());
         } else {
-            LOG.warn("No password found for user: " + userName);
+            super.handlePasswordCallback(pc);
         }
-    }
 
-    private void handleRealmCallback(RealmCallback rc) {
-        LOG.debug("handleRealmCallback: "+ rc.getDefaultText());
-        rc.setText(rc.getDefaultText());
     }
 
-    private void handleAuthorizeCallback(AuthorizeCallback ac) {
-        String authenticationID = ac.getAuthenticationID();
-        LOG.info("Successfully authenticated client: authenticationID = " + authenticationID + " authorizationID = " + ac.getAuthorizationID());
-
-        //if authorizationId is not set, set it to authenticationId.
-        if(ac.getAuthorizationID() == null) {
-            ac.setAuthorizedID(authenticationID);
-        }
-
-        //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We
-        //add the authNid as the real user in reqContext's subject which will be used during authorization.
-        if(!authenticationID.equals(ac.getAuthorizationID())) {
-            LOG.info("Impersonation attempt  authenticationID = " + ac.getAuthenticationID() + " authorizationID = " + ac.getAuthorizationID());
-            ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID()));
-        } else {
-            ReqContext.context().setRealPrincipal(null);
-        }
-
-        ac.setAuthorized(true);
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
index 25c7609..1350bdf 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
@@ -17,64 +17,15 @@
  */
 package org.apache.storm.security.auth.plain;
 
-import java.io.IOException;
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.CallbackHandler;
-import javax.security.auth.callback.NameCallback;
-import javax.security.auth.callback.PasswordCallback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.RealmCallback;
+import org.apache.storm.security.auth.AbstractSaslClientCallbackHandler;
 
+public class PlainClientCallbackHandler extends AbstractSaslClientCallbackHandler {
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *  client side callback handler.
- */
-public class PlainClientCallbackHandler implements CallbackHandler {
-    private static final String USERNAME = "username";
-    private static final String PASSWORD = "password";
-    private static final Logger LOG = LoggerFactory.getLogger(PlainClientCallbackHandler.class);
-    private String _username = "username";
-    private String _password = "password";
-
-    /**
-     * This method is invoked by SASL for authentication challenges
-     * @param callbacks a collection of challenge callbacks 
+    /*
+     * For plain, using constants for a pair of user name and password.
      */
-    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
-        for (Callback c : callbacks) {
-            if (c instanceof NameCallback) {
-                LOG.debug("name callback");
-                NameCallback nc = (NameCallback) c;
-                nc.setName(_username);
-            } else if (c instanceof PasswordCallback) {
-                LOG.debug("password callback");
-                PasswordCallback pc = (PasswordCallback)c;
-                if (_password != null) {
-                    pc.setPassword(_password.toCharArray());
-                } 
-            } else if (c instanceof AuthorizeCallback) {
-                LOG.debug("authorization callback");
-                AuthorizeCallback ac = (AuthorizeCallback) c;
-                String authid = ac.getAuthenticationID();
-                String authzid = ac.getAuthorizationID();
-                if (authid.equals(authzid)) {
-                    ac.setAuthorized(true);
-                } else {
-                    ac.setAuthorized(false);
-                }
-                if (ac.isAuthorized()) {
-                    ac.setAuthorizedID(authzid);
-                }
-            } else if (c instanceof RealmCallback) {
-                RealmCallback rc = (RealmCallback) c;
-                ((RealmCallback) c).setText(rc.getDefaultText());
-            } else {
-                throw new UnsupportedCallbackException(c);
-            }
-        }
+    public PlainClientCallbackHandler() {
+        _username = USERNAME;
+        _password = PASSWORD;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
index facc352..211a4b7 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
@@ -19,14 +19,8 @@ package org.apache.storm.security.auth.plain;
 
 import org.apache.storm.security.auth.AuthUtils;
 import org.apache.storm.security.auth.SaslTransportPlugin;
-import org.apache.storm.utils.ExtendedThreadPoolExecutor;
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TThreadPoolServer;
 import org.apache.thrift.transport.TSaslClientTransport;
 import org.apache.thrift.transport.TSaslServerTransport;
-import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.transport.TTransportFactory;
@@ -36,11 +30,6 @@ import org.slf4j.LoggerFactory;
 import javax.security.auth.callback.CallbackHandler;
 import java.io.IOException;
 import java.security.Security;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
 public class PlainSaslTransportPlugin extends SaslTransportPlugin {
     public static final String PLAIN = "PLAIN";
@@ -49,11 +38,11 @@ public class PlainSaslTransportPlugin extends SaslTransportPlugin {
     @Override
     protected TTransportFactory getServerTransportFactory() throws IOException {
         //create an authentication callback handler
-        CallbackHandler serer_callback_handler = new PlainServerCallbackHandler();
+        CallbackHandler server_callback_handler = new PlainServerCallbackHandler();
         Security.addProvider(new SaslPlainServer.SecurityProvider());
         //create a transport factory that will invoke our auth callback for digest
         TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory();
-        factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, serer_callback_handler);
+        factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, server_callback_handler);
 
         LOG.info("SASL PLAIN transport factory will be used");
         return factory;

http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
index e1ae2d9..da16825 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.storm.security.auth.AbstractSaslServerCallbackHandler;
 import org.apache.storm.security.auth.ReqContext;
 import org.apache.storm.security.auth.SaslTransportPlugin;
 import org.slf4j.Logger;
@@ -37,72 +38,11 @@ import javax.security.sasl.RealmCallback;
 /**
  * SASL server side callback handler
  */
-public class PlainServerCallbackHandler implements CallbackHandler {
-    private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class);
-    private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword";
-
-    private String userName="username";
-    private final Map<String,String> credentials = new HashMap<>();
+public class PlainServerCallbackHandler extends AbstractSaslServerCallbackHandler {
 
     public PlainServerCallbackHandler() throws IOException {
+        userName="username";
         credentials.put("username", "password");
     }
 
-    public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
-        for (Callback callback : callbacks) {
-            if (callback instanceof NameCallback) {
-                handleNameCallback((NameCallback) callback);
-            } else if (callback instanceof PasswordCallback) {
-                handlePasswordCallback((PasswordCallback) callback);
-            } else if (callback instanceof RealmCallback) {
-                handleRealmCallback((RealmCallback) callback);
-            } else if (callback instanceof AuthorizeCallback) {
-                handleAuthorizeCallback((AuthorizeCallback) callback);
-            }
-        }
-    }
-
-    private void handleNameCallback(NameCallback nc) {
-        LOG.debug("handleNameCallback");
-        userName = nc.getDefaultName();
-        nc.setName(nc.getDefaultName());
-    }
-
-    private void handlePasswordCallback(PasswordCallback pc) {
-        LOG.debug("handlePasswordCallback");
-        if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) {
-            // superuser: use Java system property for password, if available.
-            pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray());
-        } else if (credentials.containsKey(userName) ) {
-            pc.setPassword(credentials.get(userName).toCharArray());
-        } else {
-            LOG.warn("No password found for user: " + userName);
-        }
-    }
-
-    private void handleRealmCallback(RealmCallback rc) {
-        LOG.debug("handleRealmCallback: "+ rc.getDefaultText());
-        rc.setText(rc.getDefaultText());
-    }
-
-    private void handleAuthorizeCallback(AuthorizeCallback ac) {
-        String authenticationID = ac.getAuthenticationID();
-        LOG.info("Successfully authenticated client: authenticationID = " + authenticationID + " authorizationID = " + ac.getAuthorizationID());
-
-        //if authorizationId is not set, set it to authenticationId.
-        if(ac.getAuthorizationID() == null) {
-            ac.setAuthorizedID(authenticationID);
-        }
-
-        //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We
-        //add the authNid as the real user in reqContext's subject which will be used during authorization.
-        if(!authenticationID.equals(ac.getAuthorizationID())) {
-            LOG.info("Impersonation attempt  authenticationID = " + ac.getAuthenticationID() + " authorizationID = " + ac.getAuthorizationID());
-            ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID()));
-        } else {
-            ReqContext.context().setRealPrincipal(null);
-        }
-
-        ac.setAuthorized(true);
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/cccb9766/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
index a76c481..dd2582c 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.storm.security.auth.plain;
 
-import java.security.Provider;
-import java.util.Map;
-
-import javax.security.auth.callback.*;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
 import javax.security.sasl.AuthorizeCallback;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslServer;
 import javax.security.sasl.SaslServerFactory;
+import java.security.Provider;
+import java.util.Map;
 
 public class SaslPlainServer implements SaslServer {
   @SuppressWarnings("serial")
@@ -95,7 +96,7 @@ public class SaslPlainServer implements SaslServer {
       PasswordCallback pc = new PasswordCallback("SASL PLAIN", false);
       pc.setPassword(parts[2].toCharArray());
       AuthorizeCallback ac = new AuthorizeCallback(parts[1], parts[0]);
-      cbh.handle(new Callback[]{nc, pc, ac});      
+      cbh.handle(new Callback[]{nc, pc, ac});
       if (ac.isAuthorized()) {
         authz = ac.getAuthorizedID();
       }


[7/7] storm git commit: Added STORM-1469 to Changelog

Posted by bo...@apache.org.
Added STORM-1469 to Changelog


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

Branch: refs/heads/master
Commit: 974274372129f357fef12fcc82615931cfc8104c
Parents: 2c7e946
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Mar 7 14:09:52 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Mar 7 14:09:52 2016 -0600

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/97427437/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 021312c..a3ce384 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -49,6 +49,7 @@
  * STORM-1521: When using Kerberos login from keytab with multiple bolts/executors ticket is not renewed in hbase bolt.
 
 ## 1.0.0
+ * STORM-1469: Adding Plain Sasl Transport Plugin
  * STORM-1588: Do not add event logger details if number of event loggers is zero
  * STORM-1606: print the information of testcase which is on failure
  * STORM-1601: Check if /backpressure/storm-id node exists before requesting children


[6/7] storm git commit: Merge branch 'storm1469' of https://github.com/kishorvpatil/incubator-storm into STORM-1469

Posted by bo...@apache.org.
Merge branch 'storm1469' of https://github.com/kishorvpatil/incubator-storm into STORM-1469

STORM-1469: Adding Plain Sasl Transport Plugin


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/2c7e946f
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/2c7e946f
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/2c7e946f

Branch: refs/heads/master
Commit: 2c7e946f72fdd984f1162776486dcc614c420078
Parents: 4117fe5 1dbdfb1
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Mar 7 13:58:03 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Mar 7 13:58:03 2016 -0600

----------------------------------------------------------------------
 .../auth/AbstractSaslClientCallbackHandler.java |  76 +++++++++
 .../auth/AbstractSaslServerCallbackHandler.java |  94 +++++++++++
 .../auth/digest/ClientCallbackHandler.java      |  60 +------
 .../auth/digest/ServerCallbackHandler.java      |  61 +------
 .../auth/plain/PlainClientCallbackHandler.java  |  31 ++++
 .../auth/plain/PlainSaslTransportPlugin.java    |  71 +++++++++
 .../auth/plain/PlainServerCallbackHandler.java  |  55 +++++++
 .../security/auth/plain/SaslPlainServer.java    | 158 +++++++++++++++++++
 8 files changed, 497 insertions(+), 109 deletions(-)
----------------------------------------------------------------------



[4/7] storm git commit: Using real user-id in the ReqContext instead of username in PlainSaslTransportPlugin

Posted by bo...@apache.org.
Using real user-id in the ReqContext instead of username in PlainSaslTransportPlugin


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/5e2d44df
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/5e2d44df
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/5e2d44df

Branch: refs/heads/master
Commit: 5e2d44df8c342d29d723ac4ac90d0e1efb6884bb
Parents: 4e04ce8
Author: Kishor Patil <kp...@yahoo-inc.com>
Authored: Wed Mar 2 12:32:00 2016 -0600
Committer: Kishor Patil <kp...@yahoo-inc.com>
Committed: Wed Mar 2 12:32:00 2016 -0600

----------------------------------------------------------------------
 .../security/auth/plain/PlainClientCallbackHandler.java  |  2 +-
 .../security/auth/plain/PlainServerCallbackHandler.java  | 11 +++++++++--
 2 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5e2d44df/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
index 1350bdf..13340df 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java
@@ -25,7 +25,7 @@ public class PlainClientCallbackHandler extends AbstractSaslClientCallbackHandle
      * For plain, using constants for a pair of user name and password.
      */
     public PlainClientCallbackHandler() {
-        _username = USERNAME;
+        _username = System.getProperty("user.name");
         _password = PASSWORD;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/5e2d44df/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
index da16825..c646fc9 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java
@@ -39,10 +39,17 @@ import javax.security.sasl.RealmCallback;
  * SASL server side callback handler
  */
 public class PlainServerCallbackHandler extends AbstractSaslServerCallbackHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class);
+    public static final String PASSWORD = "password";
 
     public PlainServerCallbackHandler() throws IOException {
-        userName="username";
-        credentials.put("username", "password");
+        userName=null;
+    }
+
+    protected void handlePasswordCallback(PasswordCallback pc) {
+        LOG.debug("handlePasswordCallback");
+        pc.setPassword(PASSWORD.toCharArray());
+
     }
 
 }


[5/7] storm git commit: Renaming local variable to camelCase

Posted by bo...@apache.org.
Renaming local variable to camelCase


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/1dbdfb17
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/1dbdfb17
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/1dbdfb17

Branch: refs/heads/master
Commit: 1dbdfb1769979a8391348b5275bfd4bd2a4edf18
Parents: 5e2d44d
Author: Kishor Patil <kp...@yahoo-inc.com>
Authored: Wed Mar 2 12:48:48 2016 -0600
Committer: Kishor Patil <kp...@yahoo-inc.com>
Committed: Wed Mar 2 12:48:48 2016 -0600

----------------------------------------------------------------------
 .../storm/security/auth/plain/PlainSaslTransportPlugin.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1dbdfb17/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
index 6247fe6..eaef91a 100644
--- a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
+++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
@@ -38,13 +38,13 @@ public class PlainSaslTransportPlugin extends SaslTransportPlugin {
     @Override
     protected TTransportFactory getServerTransportFactory() throws IOException {
         //create an authentication callback handler
-        CallbackHandler server_callback_handler = new PlainServerCallbackHandler();
+        CallbackHandler serverCallbackHandler = new PlainServerCallbackHandler();
         if (Security.getProvider(SaslPlainServer.SecurityProvider.SASL_PLAIN_SERVER) == null) {
             Security.addProvider(new SaslPlainServer.SecurityProvider());
         }
         //create a transport factory that will invoke our auth callback for digest
         TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory();
-        factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, server_callback_handler);
+        factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, serverCallbackHandler);
 
         LOG.info("SASL PLAIN transport factory will be used");
         return factory;