View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.security.token;
20  
21  import java.io.IOException;
22  import java.lang.reflect.UndeclaredThrowableException;
23  import java.security.PrivilegedExceptionAction;
24  
25  import com.google.protobuf.ServiceException;
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.HConstants;
30  import org.apache.hadoop.hbase.TableName;
31  import org.apache.hadoop.hbase.client.HTable;
32  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
33  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
34  import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
35  import org.apache.hadoop.io.Text;
36  import org.apache.hadoop.mapred.JobConf;
37  import org.apache.hadoop.mapreduce.Job;
38  import org.apache.hadoop.security.UserGroupInformation;
39  import org.apache.hadoop.security.token.Token;
40  
41  /**
42   * Utility methods for obtaining authentication tokens.
43   */
44  public class TokenUtil {
45    private static Log LOG = LogFactory.getLog(TokenUtil.class);
46  
47    /**
48     * Obtain and return an authentication token for the current user.
49     * @param conf The configuration for connecting to the cluster
50     * @return the authentication token instance
51     */
52    public static Token<AuthenticationTokenIdentifier> obtainToken(
53        Configuration conf) throws IOException {
54      HTable meta = null;
55      try {
56        meta = new HTable(conf, TableName.META_TABLE_NAME);
57        CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
58        AuthenticationProtos.AuthenticationService.BlockingInterface service =
59            AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
60        AuthenticationProtos.GetAuthenticationTokenResponse response = service.getAuthenticationToken(null,
61            AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
62  
63        return ProtobufUtil.toToken(response.getToken());
64      } catch (ServiceException se) {
65        ProtobufUtil.toIOException(se);
66      } finally {
67        if (meta != null) {
68          meta.close();
69        }
70      }
71      // dummy return for ServiceException catch block
72      return null;
73    }
74  
75    private static Text getClusterId(Token<AuthenticationTokenIdentifier> token)
76        throws IOException {
77      return token.getService() != null
78          ? token.getService() : new Text("default");
79    }
80  
81    /**
82     * Obtain an authentication token for the given user and add it to the
83     * user's credentials.
84     * @param conf The configuration for connecting to the cluster
85     * @param user The user for whom to obtain the token
86     * @throws IOException If making a remote call to the {@link TokenProvider} fails
87     * @throws InterruptedException If executing as the given user is interrupted
88     */
89    public static void obtainAndCacheToken(final Configuration conf,
90        UserGroupInformation user)
91        throws IOException, InterruptedException {
92      try {
93        Token<AuthenticationTokenIdentifier> token =
94            user.doAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
95              public Token<AuthenticationTokenIdentifier> run() throws Exception {
96                return obtainToken(conf);
97              }
98            });
99  
100       if (token == null) {
101         throw new IOException("No token returned for user "+user.getUserName());
102       }
103       if (LOG.isDebugEnabled()) {
104         LOG.debug("Obtained token "+token.getKind().toString()+" for user "+
105             user.getUserName());
106       }
107       user.addToken(token);
108     } catch (IOException ioe) {
109       throw ioe;
110     } catch (InterruptedException ie) {
111       throw ie;
112     } catch (RuntimeException re) {
113       throw re;
114     } catch (Exception e) {
115       throw new UndeclaredThrowableException(e,
116           "Unexpected exception obtaining token for user "+user.getUserName());
117     }
118   }
119 
120   /**
121    * Obtain an authentication token on behalf of the given user and add it to
122    * the credentials for the given map reduce job.
123    * @param conf The configuration for connecting to the cluster
124    * @param user The user for whom to obtain the token
125    * @param job The job instance in which the token should be stored
126    * @throws IOException If making a remote call to the {@link TokenProvider} fails
127    * @throws InterruptedException If executing as the given user is interrupted
128    */
129   public static void obtainTokenForJob(final Configuration conf,
130       UserGroupInformation user, Job job)
131       throws IOException, InterruptedException {
132     try {
133       Token<AuthenticationTokenIdentifier> token =
134           user.doAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
135             public Token<AuthenticationTokenIdentifier> run() throws Exception {
136               return obtainToken(conf);
137             }
138           });
139 
140       if (token == null) {
141         throw new IOException("No token returned for user "+user.getUserName());
142       }
143       Text clusterId = getClusterId(token);
144       LOG.info("Obtained token "+token.getKind().toString()+" for user "+
145           user.getUserName() + " on cluster "+clusterId.toString());
146       job.getCredentials().addToken(clusterId, token);
147     } catch (IOException ioe) {
148       throw ioe;
149     } catch (InterruptedException ie) {
150       throw ie;
151     } catch (RuntimeException re) {
152       throw re;
153     } catch (Exception e) {
154       throw new UndeclaredThrowableException(e,
155           "Unexpected exception obtaining token for user "+user.getUserName());
156     }
157   }
158 
159   /**
160    * Obtain an authentication token on behalf of the given user and add it to
161    * the credentials for the given map reduce job.
162    * @param user The user for whom to obtain the token
163    * @param job The job configuration in which the token should be stored
164    * @throws IOException If making a remote call to the {@link TokenProvider} fails
165    * @throws InterruptedException If executing as the given user is interrupted
166    */
167   public static void obtainTokenForJob(final JobConf job,
168       UserGroupInformation user)
169       throws IOException, InterruptedException {
170     try {
171       Token<AuthenticationTokenIdentifier> token =
172           user.doAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
173             public Token<AuthenticationTokenIdentifier> run() throws Exception {
174               return obtainToken(job);
175             }
176           });
177 
178       if (token == null) {
179         throw new IOException("No token returned for user "+user.getUserName());
180       }
181       Text clusterId = getClusterId(token);
182       LOG.info("Obtained token "+token.getKind().toString()+" for user "+
183           user.getUserName()+" on cluster "+clusterId.toString());
184       job.getCredentials().addToken(clusterId, token);
185     } catch (IOException ioe) {
186       throw ioe;
187     } catch (InterruptedException ie) {
188       throw ie;
189     } catch (RuntimeException re) {
190       throw re;
191     } catch (Exception e) {
192       throw new UndeclaredThrowableException(e,
193           "Unexpected exception obtaining token for user "+user.getUserName());
194     }
195   }
196 }