[ 
https://issues.apache.org/jira/browse/STORM-346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14065605#comment-14065605
 ] 

ASF GitHub Bot commented on STORM-346:
--------------------------------------

Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/incubator-storm/pull/190#discussion_r15086078
  
    --- Diff: 
storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java ---
    @@ -0,0 +1,298 @@
    +/**
    + * 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 backtype.storm.security.auth.kerberos;
    +
    +import backtype.storm.Config;
    +import backtype.storm.security.auth.IAutoCredentials;
    +import backtype.storm.security.auth.ICredentialsRenewer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import javax.xml.bind.DatatypeConverter;
    +import java.io.*;
    +import java.lang.reflect.Method;
    +import java.net.URI;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +/**
    + * Automatically get HDFS delegation tokens and push it to user's 
topology. The class
    + * assumes that HDFS configuration files are in your class path.
    + */
    +public class AutoHDFS implements IAutoCredentials, ICredentialsRenewer {
    +    private static final Logger LOG = 
LoggerFactory.getLogger(AutoHDFS.class);
    +    public static final String HDFS_CREDENTIALS = "HDFS_CREDENTIALS";
    +    private static final String CONF_KEYTAB_KEY = "keytab";
    +    private static final String CONF_USER_KEY = "user";
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) {
    +        this.conf = conf;
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private Object getConfiguration() {
    +        try {
    +            final String hdfsUser = (String) conf.get(Config.HDFS_USER);
    +            final String hdfsUserKeyTab = (String) 
conf.get(Config.HDFS_USER_KEYTAB);
    +
    +            /**
    +             *  Configuration configuration = new Configuration();
    +             *  configuration.set(CONF_KEYTAB_KEY, hdfsUserKeyTab);
    +             *  configuration.set(CONF_USER_KEY, hdfsUser);
    +             */
    +            Class configurationClass = 
Class.forName("org.apache.hadoop.conf.Configuration");
    +            Object configuration = configurationClass.newInstance();
    +
    +            Method setMethod = configurationClass.getMethod("set", 
String.class, String.class);
    +            setMethod.invoke(configuration, CONF_KEYTAB_KEY, 
hdfsUserKeyTab);
    +            setMethod.invoke(configuration, CONF_USER_KEY, hdfsUser);
    +            /**
    +             * Following are the minimum set of configuration that needs 
to be set,  users should have hdfs-site.xml
    +             * and core-site.xml in the class path which should set these 
configuration.
    +             * setMethod.invoke(configuration, 
"hadoop.security.authentication", "KERBEROS");
    +             * 
setMethod.invoke(configuration,"dfs.namenode.kerberos.principal",
    +             *                                
"hdfs/[email protected]");
    +             * setMethod.invoke(configuration, 
"hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002");
    +             */
    +
    +            setMethod.invoke(configuration, 
"hadoop.security.authentication", "KERBEROS");
    +            setMethod.invoke(configuration, 
"dfs.namenode.kerberos.principal","hdfs/[email protected]");
    +            setMethod.invoke(configuration, 
"hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002");
    +
    +            //UserGroupInformation.setConfiguration(configuration);
    +            final Class ugiClass = 
Class.forName("org.apache.hadoop.security.UserGroupInformation");
    +            Method setConfigurationMethod = 
ugiClass.getMethod("setConfiguration", configurationClass);
    +            setConfigurationMethod.invoke(null, configuration);
    +            return configuration;
    +        }  catch (Exception e) {
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private void login(Object configuration) {
    +        try {
    +            Class configurationClass = 
Class.forName("org.apache.hadoop.conf.Configuration");
    +            final Class securityUtilClass = 
Class.forName("org.apache.hadoop.security.SecurityUtil");
    +            Method loginMethod = securityUtilClass.getMethod("login", 
configurationClass, String.class, String.class);
    +            loginMethod.invoke(null, configuration, CONF_KEYTAB_KEY, 
CONF_USER_KEY);
    +        } catch (Exception e) {
    +           throw new RuntimeException("Failed to login to hdfs .", e);
    +        }
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private byte[] getHDFSCredsWithDelegationToken() throws Exception {
    +
    +        try {
    +            /**
    +             * What we want to do is following:
    +             *  Configuration configuration = new Configuration();
    +             *  configuration.set(CONF_KEYTAB_KEY, hdfsUserKeyTab);
    +             *  configuration.set(CONF_USER_KEY, hdfsUser);
    +             *  UserGroupInformation.setConfiguration(configuration);
    +             *  if(UserGroupInformation.isSecurityEnabled) {
    +             *      SecurityUtil.login(configuration, CONF_KEYTAB_KEY, 
CONF_USER_KEY);
    +             *      FileSystem fs = FileSystem.get(nameNodeURI, 
configuration, topologySubmitterUser);
    +             *      UserGroupInformation ugi = 
UserGroupInformation.getCurrentUser();
    +             *      UserGroupInformation proxyUser = 
UserGroupInformation.createProxyUser(topologySubmitterUser, ugi);
    +             *      Credentials credential= proxyUser.getCredentials();
    +             *      fs.addDelegationToken(hdfsUser, credential);
    +             * }
    +             * and then return the credential object as a bytearray.
    +             */
    +            Object configuration = getConfiguration();
    +            final Class ugiClass = 
Class.forName("org.apache.hadoop.security.UserGroupInformation");
    +            final Method isSecurityEnabledMethod = 
ugiClass.getDeclaredMethod("isSecurityEnabled");
    +            boolean isSecurityEnabled = 
(Boolean)isSecurityEnabledMethod.invoke(null);
    +            if(isSecurityEnabled) {
    +                login(configuration);
    +
    +                final URI nameNodeURI = URI.create((String) 
conf.get(Config.HDFS_NAMENODE_URL));
    +                final String topologySubmitterUser = (String) 
conf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +                final String hdfsUser = (String) 
conf.get(Config.HDFS_USER);
    +
    +                Class configurationClass = 
Class.forName("org.apache.hadoop.conf.Configuration");
    +
    +                //FileSystem fs = FileSystem.get(nameNodeURI, 
configuration, topologySubmitterUser);
    +                Class fileSystemClass = 
Class.forName("org.apache.hadoop.fs.FileSystem");
    +                Method getMethod = fileSystemClass.getMethod("get", 
URI.class, configurationClass, String.class);
    +                Object fileSystem = getMethod.invoke(null, nameNodeURI, 
configuration, topologySubmitterUser);
    +
    +                //UserGroupInformation ugi = 
UserGroupInformation.getCurrentUser();
    +                Method getCurrentUserMethod = 
ugiClass.getMethod("getCurrentUser");
    +                final Object ugi = getCurrentUserMethod.invoke(null);
    +
    +                //UserGroupInformation proxyUser = 
UserGroupInformation.createProxyUser(topologySubmitterUser, ugi);
    +                Method createProxyUserMethod = 
ugiClass.getMethod("createProxyUser", String.class, ugiClass);
    +                Object proxyUGI = createProxyUserMethod.invoke(null, 
topologySubmitterUser, ugi);
    +
    +                //Credentials credential= proxyUser.getCredentials();
    +                Method getCredentialsMethod = 
ugiClass.getMethod("getCredentials");
    +                Object credentials = getCredentialsMethod.invoke(proxyUGI);
    +
    +                //fs.addDelegationToken(hdfsUser, credential);
    +                Class credentialClass = 
Class.forName("org.apache.hadoop.security.Credentials");
    +                Method addDelegationTokensMethod = 
fileSystemClass.getMethod("addDelegationTokens", String.class,
    +                        credentialClass);
    +                addDelegationTokensMethod.invoke(fileSystem, hdfsUser, 
credentials);
    +
    +
    +                ByteArrayOutputStream bao = new ByteArrayOutputStream();
    +                ObjectOutputStream out = new ObjectOutputStream(bao);
    +                Method writeMethod = credentialClass.getMethod("write", 
DataOutput.class);
    +                writeMethod.invoke(credentials, out);
    +                out.flush();
    +                out.close();
    +
    +                LOG.info(bao.toString());
    +                return bao.toByteArray();
    +            } else {
    +                throw new RuntimeException("Security is not enabled for 
HDFS");
    +            }
    +        } catch (Exception ex) {
    +            throw new RuntimeException("Failed to get delegation tokens." 
, ex);
    +        }
    +    }
    +
    +    @Override
    +    public void populateCredentials(Map<String, String> credentials) {
    --- End diff --
    
    Populate credentials is called on the gateway by the end user.  As such 
there is no reason to use a proxyUser get a delegation token, because I can 
just get it.  In fact the proxyUser should fail, because regular users are not 
authorized to act as proxy users.
    
    The idea was to change some of how nimbus used an ICredentialsRenewer, or 
have a different interface so that nimbus can ask AutoHDFS to populate the 
credentials at the very beginning. I'll try to explain better in the more 
general comments.


> (Security) Oozie style delegation tokens for HDFS/HBase
> -------------------------------------------------------
>
>                 Key: STORM-346
>                 URL: https://issues.apache.org/jira/browse/STORM-346
>             Project: Apache Storm (Incubating)
>          Issue Type: Bug
>            Reporter: Robert Joseph Evans
>            Assignee: Parth Brahmbhatt
>              Labels: security
>
> Oozie has the ability to fetch delegation tokens on behalf of other users by 
> running as a super user that can become a proxy user for almost anyone else.
> We should build one or more classes similar to AutoTGT that can fetch a 
> delegation token for HDFS/HBase, renew the token if needed, and then once the 
> token is about to permanently expire fetch a new one.
> According to some people I have talked with HBase may need to have a JIRA 
> filed against it so that it can pick up a new delegation token without 
> needing to restart the process.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to