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