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