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