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.access;
20
21 import com.google.protobuf.RpcCallback;
22 import com.google.protobuf.RpcController;
23 import com.google.protobuf.Service;
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.classification.InterfaceAudience;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.FileStatus;
29 import org.apache.hadoop.fs.FileSystem;
30 import org.apache.hadoop.fs.Path;
31 import org.apache.hadoop.fs.permission.FsPermission;
32 import org.apache.hadoop.hbase.Coprocessor;
33 import org.apache.hadoop.hbase.CoprocessorEnvironment;
34 import org.apache.hadoop.hbase.TableName;
35 import org.apache.hadoop.hbase.DoNotRetryIOException;
36 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
37 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
38 import org.apache.hadoop.hbase.ipc.RequestContext;
39 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
40 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
41 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
42 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService;
43 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
44 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadResponse;
45 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest;
46 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse;
47 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest;
48 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
49 import org.apache.hadoop.hbase.regionserver.HRegion;
50 import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
51 import org.apache.hadoop.hbase.security.User;
52 import org.apache.hadoop.hbase.security.UserProvider;
53 import org.apache.hadoop.hbase.util.Bytes;
54 import org.apache.hadoop.hbase.util.Methods;
55 import org.apache.hadoop.hbase.util.Pair;
56 import org.apache.hadoop.io.Text;
57 import org.apache.hadoop.security.UserGroupInformation;
58 import org.apache.hadoop.security.token.Token;
59
60 import java.io.IOException;
61 import java.math.BigInteger;
62 import java.security.PrivilegedAction;
63 import java.security.SecureRandom;
64 import java.util.ArrayList;
65 import java.util.List;
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93 @InterfaceAudience.Private
94 public class SecureBulkLoadEndpoint extends SecureBulkLoadService
95 implements CoprocessorService, Coprocessor {
96
97 public static final long VERSION = 0L;
98
99
100 private static final int RANDOM_WIDTH = 320;
101 private static final int RANDOM_RADIX = 32;
102
103 private static Log LOG = LogFactory.getLog(SecureBulkLoadEndpoint.class);
104
105 private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
106 private final static FsPermission PERM_HIDDEN = FsPermission.valueOf("-rwx--x--x");
107
108 private SecureRandom random;
109 private FileSystem fs;
110 private Configuration conf;
111
112
113
114 private Path baseStagingDir;
115
116 private RegionCoprocessorEnvironment env;
117
118 private UserProvider userProvider;
119
120 @Override
121 public void start(CoprocessorEnvironment env) {
122 this.env = (RegionCoprocessorEnvironment)env;
123 random = new SecureRandom();
124 conf = env.getConfiguration();
125 baseStagingDir = SecureBulkLoadUtil.getBaseStagingDir(conf);
126 this.userProvider = UserProvider.instantiate(conf);
127
128 try {
129 fs = FileSystem.get(conf);
130 fs.mkdirs(baseStagingDir, PERM_HIDDEN);
131 fs.setPermission(baseStagingDir, PERM_HIDDEN);
132
133 fs.mkdirs(new Path(baseStagingDir,"DONOTERASE"), PERM_HIDDEN);
134 FileStatus status = fs.getFileStatus(baseStagingDir);
135 if(status == null) {
136 throw new IllegalStateException("Failed to create staging directory");
137 }
138 if(!status.getPermission().equals(PERM_HIDDEN)) {
139 throw new IllegalStateException(
140 "Directory already exists but permissions aren't set to '-rwx--x--x' ");
141 }
142 } catch (IOException e) {
143 throw new IllegalStateException("Failed to get FileSystem instance",e);
144 }
145 }
146
147 @Override
148 public void stop(CoprocessorEnvironment env) throws IOException {
149 }
150
151 @Override
152 public void prepareBulkLoad(RpcController controller,
153 PrepareBulkLoadRequest request,
154 RpcCallback<PrepareBulkLoadResponse> done){
155 try {
156 getAccessController().prePrepareBulkLoad(env);
157 String bulkToken = createStagingDir(baseStagingDir,
158 getActiveUser(), ProtobufUtil.toTableName(request.getTableName())).toString();
159 done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build());
160 } catch (IOException e) {
161 ResponseConverter.setControllerException(controller, e);
162 }
163 done.run(null);
164 }
165
166 @Override
167 public void cleanupBulkLoad(RpcController controller,
168 CleanupBulkLoadRequest request,
169 RpcCallback<CleanupBulkLoadResponse> done) {
170 try {
171 getAccessController().preCleanupBulkLoad(env);
172 fs.delete(createStagingDir(baseStagingDir,
173 getActiveUser(),
174 env.getRegion().getTableDesc().getTableName(),
175 new Path(request.getBulkToken()).getName()),
176 true);
177 done.run(CleanupBulkLoadResponse.newBuilder().build());
178 } catch (IOException e) {
179 ResponseConverter.setControllerException(controller, e);
180 }
181 done.run(null);
182 }
183
184 @Override
185 public void secureBulkLoadHFiles(RpcController controller,
186 SecureBulkLoadHFilesRequest request,
187 RpcCallback<SecureBulkLoadHFilesResponse> done) {
188 final List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
189 for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
190 familyPaths.add(new Pair(el.getFamily().toByteArray(),el.getPath()));
191 }
192 final Token userToken =
193 new Token(request.getFsToken().getIdentifier().toByteArray(),
194 request.getFsToken().getPassword().toByteArray(),
195 new Text(request.getFsToken().getKind()),
196 new Text(request.getFsToken().getService()));
197 final String bulkToken = request.getBulkToken();
198 User user = getActiveUser();
199 final UserGroupInformation ugi = user.getUGI();
200 if(userToken != null) {
201 ugi.addToken(userToken);
202 } else if (userProvider.isHadoopSecurityEnabled()) {
203
204
205 ResponseConverter.setControllerException(controller,
206 new DoNotRetryIOException("User token cannot be null"));
207 return;
208 }
209
210 HRegion region = env.getRegion();
211 boolean bypass = false;
212 if (region.getCoprocessorHost() != null) {
213 try {
214 bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
215 } catch (IOException e) {
216 ResponseConverter.setControllerException(controller, e);
217 done.run(null);
218 return;
219 }
220 }
221 boolean loaded = false;
222 if (!bypass) {
223 loaded = ugi.doAs(new PrivilegedAction<Boolean>() {
224 @Override
225 public Boolean run() {
226 FileSystem fs = null;
227 try {
228 Configuration conf = env.getConfiguration();
229 fs = FileSystem.get(conf);
230 for(Pair<byte[], String> el: familyPaths) {
231 Path p = new Path(el.getSecond());
232 LOG.trace("Setting permission for: " + p);
233 fs.setPermission(p, PERM_ALL_ACCESS);
234 Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
235 if(!fs.exists(stageFamily)) {
236 fs.mkdirs(stageFamily);
237 fs.setPermission(stageFamily, PERM_ALL_ACCESS);
238 }
239 }
240
241
242 return env.getRegion().bulkLoadHFiles(familyPaths, true,
243 new SecureBulkLoadListener(fs, bulkToken));
244 } catch (Exception e) {
245 LOG.error("Failed to complete bulk load", e);
246 }
247 return false;
248 }
249 });
250 }
251 if (region.getCoprocessorHost() != null) {
252 try {
253 loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
254 } catch (IOException e) {
255 ResponseConverter.setControllerException(controller, e);
256 done.run(null);
257 return;
258 }
259 }
260 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build());
261 }
262
263 private AccessController getAccessController() {
264 return (AccessController) this.env.getRegion()
265 .getCoprocessorHost().findCoprocessor(AccessController.class.getName());
266 }
267
268 private Path createStagingDir(Path baseDir,
269 User user,
270 TableName tableName) throws IOException {
271 String randomDir = user.getShortName()+"__"+ tableName +"__"+
272 (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
273 return createStagingDir(baseDir, user, tableName, randomDir);
274 }
275
276 private Path createStagingDir(Path baseDir,
277 User user,
278 TableName tableName,
279 String randomDir) throws IOException {
280 Path p = new Path(baseDir, randomDir);
281 fs.mkdirs(p, PERM_ALL_ACCESS);
282 fs.setPermission(p, PERM_ALL_ACCESS);
283 return p;
284 }
285
286 private User getActiveUser() {
287 User user = RequestContext.getRequestUser();
288 if (!RequestContext.isInRequestContext()) {
289 return null;
290 }
291
292
293 if("simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
294 return User.createUserForTesting(conf, user.getShortName(), new String[]{});
295 }
296
297 return user;
298 }
299
300 @Override
301 public Service getService() {
302 return this;
303 }
304
305 private static class SecureBulkLoadListener implements HRegion.BulkLoadListener {
306 private FileSystem fs;
307 private String stagingDir;
308
309 public SecureBulkLoadListener(FileSystem fs, String stagingDir) {
310 this.fs = fs;
311 this.stagingDir = stagingDir;
312 }
313
314 @Override
315 public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
316 Path p = new Path(srcPath);
317 Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
318
319 if(!isFile(p)) {
320 throw new IOException("Path does not reference a file: " + p);
321 }
322
323 LOG.debug("Moving " + p + " to " + stageP);
324 if(!fs.rename(p, stageP)) {
325 throw new IOException("Failed to move HFile: " + p + " to " + stageP);
326 }
327 return stageP.toString();
328 }
329
330 @Override
331 public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
332 LOG.debug("Bulk Load done for: " + srcPath);
333 }
334
335 @Override
336 public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
337 Path p = new Path(srcPath);
338 Path stageP = new Path(stagingDir,
339 new Path(Bytes.toString(family), p.getName()));
340 LOG.debug("Moving " + stageP + " back to " + p);
341 if(!fs.rename(stageP, p))
342 throw new IOException("Failed to move HFile: " + stageP + " to " + p);
343 }
344
345
346
347
348
349
350
351
352 private boolean isFile(Path p) throws IOException {
353 FileStatus status = fs.getFileStatus(p);
354 boolean isFile = !status.isDir();
355 try {
356 isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
357 } catch (Exception e) {
358 }
359 return isFile;
360 }
361 }
362 }