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.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.HConstants;
31  import org.apache.hadoop.hbase.TableName;
32  import org.apache.hadoop.hbase.client.HTable;
33  import org.apache.hadoop.hbase.client.Table;
34  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
35  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
36  import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
37  import org.apache.hadoop.io.Text;
38  import org.apache.hadoop.mapred.JobConf;
39  import org.apache.hadoop.mapreduce.Job;
40  import org.apache.hadoop.security.UserGroupInformation;
41  import org.apache.hadoop.security.token.Token;
42  
43  /**
44   * Utility methods for obtaining authentication tokens.
45   */
46  @InterfaceAudience.Private
47  public class TokenUtil {
48    private static Log LOG = LogFactory.getLog(TokenUtil.class);
49  
50    /**
51     * Obtain and return an authentication token for the current user.
52     * @param conf The configuration for connecting to the cluster
53     * @return the authentication token instance
54     */
55    public static Token<AuthenticationTokenIdentifier> obtainToken(
56        Configuration conf) throws IOException {
57      Table meta = null;
58      try {
59        meta = new HTable(conf, TableName.META_TABLE_NAME);
60        CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
61        AuthenticationProtos.AuthenticationService.BlockingInterface service =
62            AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
63        AuthenticationProtos.GetAuthenticationTokenResponse response = service.getAuthenticationToken(null,
64            AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
65  
66        return ProtobufUtil.toToken(response.getToken());
67      } catch (ServiceException se) {
68        ProtobufUtil.toIOException(se);
69      } finally {
70        if (meta != null) {
71          meta.close();
72        }
73      }
74      // dummy return for ServiceException catch block
75      return null;
76    }
77  
78    private static Text getClusterId(Token<AuthenticationTokenIdentifier> token)
79        throws IOException {
80      return token.getService() != null
81          ? token.getService() : new Text("default");
82    }
83  
84    /**
85     * Obtain an authentication token for the given user and add it to the
86     * user's credentials.
87     * @param conf The configuration for connecting to the cluster
88     * @param user The user for whom to obtain the token
89     * @throws IOException If making a remote call to the {@link TokenProvider} fails
90     * @throws InterruptedException If executing as the given user is interrupted
91     */
92    public static void obtainAndCacheToken(final Configuration conf,
93        UserGroupInformation user)
94        throws IOException, InterruptedException {
95      try {
96        Token<AuthenticationTokenIdentifier> token =
97            user.doAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
98              public Token<AuthenticationTokenIdentifier> run() throws Exception {
99                return obtainToken(conf);
100             }
101           });
102 
103       if (token == null) {
104         throw new IOException("No token returned for user "+user.getUserName());
105       }
106       if (LOG.isDebugEnabled()) {
107         LOG.debug("Obtained token "+token.getKind().toString()+" for user "+
108             user.getUserName());
109       }
110       user.addToken(token);
111     } catch (IOException ioe) {
112       throw ioe;
113     } catch (InterruptedException ie) {
114       throw ie;
115     } catch (RuntimeException re) {
116       throw re;
117     } catch (Exception e) {
118       throw new UndeclaredThrowableException(e,
119           "Unexpected exception obtaining token for user "+user.getUserName());
120     }
121   }
122 
123   /**
124    * Obtain an authentication token on behalf of the given user and add it to
125    * the credentials for the given map reduce job.
126    * @param conf The configuration for connecting to the cluster
127    * @param user The user for whom to obtain the token
128    * @param job The job instance in which the token should be stored
129    * @throws IOException If making a remote call to the {@link TokenProvider} fails
130    * @throws InterruptedException If executing as the given user is interrupted
131    */
132   public static void obtainTokenForJob(final Configuration conf,
133       UserGroupInformation user, Job job)
134       throws IOException, InterruptedException {
135     try {
136       Token<AuthenticationTokenIdentifier> token =
137           user.doAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
138             public Token<AuthenticationTokenIdentifier> run() throws Exception {
139               return obtainToken(conf);
140             }
141           });
142 
143       if (token == null) {
144         throw new IOException("No token returned for user "+user.getUserName());
145       }
146       Text clusterId = getClusterId(token);
147       LOG.info("Obtained token "+token.getKind().toString()+" for user "+
148           user.getUserName() + " on cluster "+clusterId.toString());
149       job.getCredentials().addToken(clusterId, token);
150     } catch (IOException ioe) {
151       throw ioe;
152     } catch (InterruptedException ie) {
153       throw ie;
154     } catch (RuntimeException re) {
155       throw re;
156     } catch (Exception e) {
157       throw new UndeclaredThrowableException(e,
158           "Unexpected exception obtaining token for user "+user.getUserName());
159     }
160   }
161 
162   /**
163    * Obtain an authentication token on behalf of the given user and add it to
164    * the credentials for the given map reduce job.
165    * @param user The user for whom to obtain the token
166    * @param job The job configuration in which the token should be stored
167    * @throws IOException If making a remote call to the {@link TokenProvider} fails
168    * @throws InterruptedException If executing as the given user is interrupted
169    */
170   public static void obtainTokenForJob(final JobConf job,
171       UserGroupInformation user)
172       throws IOException, InterruptedException {
173     try {
174       Token<AuthenticationTokenIdentifier> token =
175           user.doAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
176             public Token<AuthenticationTokenIdentifier> run() throws Exception {
177               return obtainToken(job);
178             }
179           });
180 
181       if (token == null) {
182         throw new IOException("No token returned for user "+user.getUserName());
183       }
184       Text clusterId = getClusterId(token);
185       LOG.info("Obtained token "+token.getKind().toString()+" for user "+
186           user.getUserName()+" on cluster "+clusterId.toString());
187       job.getCredentials().addToken(clusterId, token);
188     } catch (IOException ioe) {
189       throw ioe;
190     } catch (InterruptedException ie) {
191       throw ie;
192     } catch (RuntimeException re) {
193       throw re;
194     } catch (Exception e) {
195       throw new UndeclaredThrowableException(e,
196           "Unexpected exception obtaining token for user "+user.getUserName());
197     }
198   }
199 }