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
25 import org.apache.commons.lang.StringUtils;
26 import org.apache.commons.lang.mutable.MutableInt;
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.client.Connection;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.FileStatus;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.FileUtil;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.fs.permission.FsPermission;
37 import org.apache.hadoop.hbase.Coprocessor;
38 import org.apache.hadoop.hbase.CoprocessorEnvironment;
39 import org.apache.hadoop.hbase.TableName;
40 import org.apache.hadoop.hbase.DoNotRetryIOException;
41 import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver;
42 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
43 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
44 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
45 import org.apache.hadoop.hbase.ipc.RpcServer;
46 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
47 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
48 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
49 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService;
50 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
51 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadResponse;
52 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest;
53 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse;
54 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest;
55 import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
56 import org.apache.hadoop.hbase.regionserver.Region;
57 import org.apache.hadoop.hbase.regionserver.Region.BulkLoadListener;
58 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
59 import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
60 import org.apache.hadoop.hbase.security.User;
61 import org.apache.hadoop.hbase.security.UserProvider;
62 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
63 import org.apache.hadoop.hbase.security.token.TokenUtil;
64 import org.apache.hadoop.hbase.util.Bytes;
65 import org.apache.hadoop.hbase.util.FSHDFSUtils;
66 import org.apache.hadoop.hbase.util.Methods;
67 import org.apache.hadoop.hbase.util.Pair;
68 import org.apache.hadoop.io.Text;
69 import org.apache.hadoop.security.UserGroupInformation;
70 import org.apache.hadoop.security.token.Token;
71
72 import java.io.IOException;
73 import java.math.BigInteger;
74 import java.security.PrivilegedAction;
75 import java.security.SecureRandom;
76 import java.util.ArrayList;
77 import java.util.Arrays;
78 import java.util.HashMap;
79 import java.util.HashSet;
80 import java.util.List;
81 import java.util.Map;
82 import java.util.Set;
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110 @InterfaceAudience.Private
111 public class SecureBulkLoadEndpoint extends SecureBulkLoadService
112 implements CoprocessorService, Coprocessor {
113
114 public static final long VERSION = 0L;
115
116
117 private static final int RANDOM_WIDTH = 320;
118 private static final int RANDOM_RADIX = 32;
119
120 private static final Log LOG = LogFactory.getLog(SecureBulkLoadEndpoint.class);
121
122 private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
123 private final static FsPermission PERM_HIDDEN = FsPermission.valueOf("-rwx--x--x");
124
125 public static final String FS_WITHOUT_SUPPORT_PERMISSION_KEY =
126 "hbase.secure.bulkload.fs.permission.lacking";
127 public static final String FS_WITHOUT_SUPPORT_PERMISSION_DEFAULT =
128 "s3,s3a,s3n,wasb,wasbs,swift,adfs,abfs,viewfs";
129
130 private SecureRandom random;
131 private FileSystem fs;
132 private Configuration conf;
133
134
135
136 private Path baseStagingDir;
137
138 private RegionCoprocessorEnvironment env;
139 private Connection conn;
140
141 private UserProvider userProvider;
142 private static HashMap<UserGroupInformation, MutableInt> ugiReferenceCounter = new HashMap<>();
143
144 @Override
145 public void start(CoprocessorEnvironment env) {
146 this.env = (RegionCoprocessorEnvironment)env;
147 random = new SecureRandom();
148 RegionServerServices svc = ((RegionCoprocessorEnvironment)env).getRegionServerServices();
149 this.conn = svc.getConnection();
150 conf = env.getConfiguration();
151 baseStagingDir = SecureBulkLoadUtil.getBaseStagingDir(conf);
152 this.userProvider = UserProvider.instantiate(conf);
153 Set<String> fsSet = getFileSystemSchemesWithoutPermissionSupport(conf);
154
155 try {
156 fs = baseStagingDir.getFileSystem(conf);
157 if (!fs.exists(baseStagingDir)) {
158 fs.mkdirs(baseStagingDir, PERM_HIDDEN);
159 }
160 FileStatus status = fs.getFileStatus(baseStagingDir);
161 if (status == null) {
162 throw new IllegalStateException("Failed to create staging directory");
163 }
164
165
166
167 if (!status.getPermission().equals(PERM_HIDDEN)) {
168 fs.setPermission(baseStagingDir, PERM_HIDDEN);
169 status = fs.getFileStatus(baseStagingDir);
170 }
171
172
173 Path doNotEraseDir = new Path(baseStagingDir, "DONOTERASE");
174 if (!fs.exists(doNotEraseDir)) {
175 fs.mkdirs(doNotEraseDir, PERM_HIDDEN);
176 fs.setPermission(doNotEraseDir, PERM_HIDDEN);
177 }
178
179 String scheme = fs.getScheme().toLowerCase();
180 if (!fsSet.contains(scheme) && !status.getPermission().equals(PERM_HIDDEN)) {
181 throw new IllegalStateException(
182 "Staging directory of " + baseStagingDir + " already exists but permissions aren't set to '-rwx--x--x' ");
183 }
184 } catch (IOException e) {
185 throw new IllegalStateException("Failed to get FileSystem instance",e);
186 }
187 }
188
189 Set<String> getFileSystemSchemesWithoutPermissionSupport(Configuration conf) {
190 final String value = conf.get(
191 FS_WITHOUT_SUPPORT_PERMISSION_KEY, FS_WITHOUT_SUPPORT_PERMISSION_DEFAULT);
192 return new HashSet<String>(Arrays.asList(StringUtils.split(value, ',')));
193 }
194
195 @Override
196 public void stop(CoprocessorEnvironment env) throws IOException {
197 }
198
199 @Override
200 public void prepareBulkLoad(RpcController controller, PrepareBulkLoadRequest request,
201 RpcCallback<PrepareBulkLoadResponse> done) {
202 try {
203 List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
204 if (bulkLoadObservers != null) {
205 ObserverContext<RegionCoprocessorEnvironment> ctx =
206 new ObserverContext<RegionCoprocessorEnvironment>(RpcServer.getRequestUser());
207 ctx.prepare(env);
208 for (BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
209 bulkLoadObserver.prePrepareBulkLoad(ctx, request);
210 }
211 }
212 String bulkToken = createStagingDir(baseStagingDir, getActiveUser(),
213 ProtobufUtil.toTableName(request.getTableName())).toString();
214 done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build());
215 } catch (IOException e) {
216 ResponseConverter.setControllerException(controller, e);
217 }
218 done.run(null);
219 }
220
221 @Override
222 public void cleanupBulkLoad(RpcController controller, CleanupBulkLoadRequest request,
223 RpcCallback<CleanupBulkLoadResponse> done) {
224 try {
225 List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
226 if (bulkLoadObservers != null) {
227 ObserverContext<RegionCoprocessorEnvironment> ctx =
228 new ObserverContext<RegionCoprocessorEnvironment>(RpcServer.getRequestUser());
229 ctx.prepare(env);
230 for (BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
231 bulkLoadObserver.preCleanupBulkLoad(ctx, request);
232 }
233 }
234 Path path = new Path(request.getBulkToken());
235 if (!fs.delete(path, true)) {
236 if (fs.exists(path)) {
237 throw new IOException("Failed to clean up " + path);
238 }
239 }
240 LOG.info("Cleaned up " + path + " successfully.");
241 done.run(CleanupBulkLoadResponse.newBuilder().build());
242 } catch (IOException e) {
243 ResponseConverter.setControllerException(controller, e);
244 } finally {
245 UserGroupInformation ugi = getActiveUser().getUGI();
246 try {
247 if (!UserGroupInformation.getLoginUser().equals(ugi) && !isUserReferenced(ugi)) {
248 FileSystem.closeAllForUGI(ugi);
249 }
250 } catch (IOException e) {
251 LOG.error("Failed to close FileSystem for: " + ugi, e);
252 }
253 }
254 done.run(null);
255 }
256
257 interface Consumer<T> {
258 void accept(T t);
259 }
260
261 private static Consumer<Region> fsCreatedListener;
262
263 static void setFsCreatedListener(Consumer<Region> listener) {
264 fsCreatedListener = listener;
265 }
266
267 private void incrementUgiReference(UserGroupInformation ugi) {
268 synchronized (ugiReferenceCounter) {
269 final MutableInt counter = ugiReferenceCounter.get(ugi);
270 if (counter == null) {
271 ugiReferenceCounter.put(ugi, new MutableInt(1));
272 } else {
273 counter.increment();
274 }
275 }
276 }
277
278 private void decrementUgiReference(UserGroupInformation ugi) {
279 synchronized (ugiReferenceCounter) {
280 final MutableInt counter = ugiReferenceCounter.get(ugi);
281 if(counter == null || counter.intValue() <= 1) {
282 ugiReferenceCounter.remove(ugi);
283 } else {
284 counter.decrement();
285 }
286 }
287 }
288
289 private boolean isUserReferenced(UserGroupInformation ugi) {
290 synchronized (ugiReferenceCounter) {
291 final MutableInt counter = ugiReferenceCounter.get(ugi);
292 return counter != null && counter.intValue() > 0;
293 }
294 }
295
296 @Override
297 public void secureBulkLoadHFiles(RpcController controller,
298 final SecureBulkLoadHFilesRequest request,
299 RpcCallback<SecureBulkLoadHFilesResponse> done) {
300 final List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
301 for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
302 familyPaths.add(new Pair(el.getFamily().toByteArray(),el.getPath()));
303 }
304
305 Token userToken = null;
306 if (userProvider.isHadoopSecurityEnabled()) {
307 userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
308 .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
309 request.getFsToken().getService()));
310 }
311 final String bulkToken = request.getBulkToken();
312 User user = getActiveUser();
313 final UserGroupInformation ugi = user.getUGI();
314 if (userProvider.isHadoopSecurityEnabled()) {
315 try {
316 Token tok = TokenUtil.obtainToken(conn);
317 if (tok != null) {
318 boolean b = ugi.addToken(tok);
319 LOG.debug("extra token added " + tok + " ret=" + b);
320 }
321 } catch (IOException ioe) {
322 LOG.warn("unable to add token", ioe);
323 }
324 }
325 if (userToken != null) {
326 ugi.addToken(userToken);
327 } else if (userProvider.isHadoopSecurityEnabled()) {
328
329
330 ResponseConverter.setControllerException(controller,
331 new DoNotRetryIOException("User token cannot be null"));
332 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
333 return;
334 }
335
336 Region region = env.getRegion();
337 boolean bypass = false;
338 if (region.getCoprocessorHost() != null) {
339 try {
340 bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
341 } catch (IOException e) {
342 ResponseConverter.setControllerException(controller, e);
343 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
344 return;
345 }
346 }
347 boolean loaded = false;
348 if (!bypass) {
349
350
351
352
353
354 if (userProvider.isHadoopSecurityEnabled()) {
355 FsDelegationToken targetfsDelegationToken = new FsDelegationToken(userProvider, "renewer");
356 try {
357 targetfsDelegationToken.acquireDelegationToken(fs);
358 } catch (IOException e) {
359 ResponseConverter.setControllerException(controller, e);
360 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
361 return;
362 }
363 Token<?> targetFsToken = targetfsDelegationToken.getUserToken();
364 if (targetFsToken != null
365 && (userToken == null || !targetFsToken.getService().equals(userToken.getService()))) {
366 ugi.addToken(targetFsToken);
367 }
368 }
369
370 incrementUgiReference(ugi);
371 loaded = ugi.doAs(new PrivilegedAction<Boolean>() {
372 @Override
373 public Boolean run() {
374 FileSystem fs = null;
375 try {
376 Configuration conf = env.getConfiguration();
377 fs = FileSystem.get(conf);
378 for(Pair<byte[], String> el: familyPaths) {
379 Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
380 if(!fs.exists(stageFamily)) {
381 fs.mkdirs(stageFamily);
382 fs.setPermission(stageFamily, PERM_ALL_ACCESS);
383 }
384 }
385 if (fsCreatedListener != null) {
386 fsCreatedListener.accept(env.getRegion());
387 }
388
389
390 return env.getRegion().bulkLoadHFiles(familyPaths, true,
391 new SecureBulkLoadListener(fs, bulkToken, conf), request.getClusterIdsList());
392 } catch (Exception e) {
393 LOG.error("Failed to complete bulk load", e);
394 }
395 return false;
396 }
397 });
398 decrementUgiReference(ugi);
399 }
400 if (region.getCoprocessorHost() != null) {
401 try {
402 loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
403 } catch (IOException e) {
404 ResponseConverter.setControllerException(controller, e);
405 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
406 return;
407 }
408 }
409 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build());
410 }
411
412 private List<BulkLoadObserver> getBulkLoadObservers() {
413 List<BulkLoadObserver> coprocessorList =
414 this.env.getRegion().getCoprocessorHost().findCoprocessors(BulkLoadObserver.class);
415
416 return coprocessorList;
417 }
418
419 private Path createStagingDir(Path baseDir,
420 User user,
421 TableName tableName) throws IOException {
422 String tblName = tableName.getNameAsString().replace(":", "_");
423 String randomDir = user.getShortName()+"__"+ tblName +"__"+
424 (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
425 return createStagingDir(baseDir, user, randomDir);
426 }
427
428 private Path createStagingDir(Path baseDir,
429 User user,
430 String randomDir) throws IOException {
431 Path p = new Path(baseDir, randomDir);
432 fs.mkdirs(p, PERM_ALL_ACCESS);
433 fs.setPermission(p, PERM_ALL_ACCESS);
434 return p;
435 }
436
437 private User getActiveUser() {
438 User user = RpcServer.getRequestUser();
439 if (user == null) {
440 return null;
441 }
442
443
444 if (userProvider.isHadoopSecurityEnabled()
445 && "simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
446 return User.createUserForTesting(conf, user.getShortName(), new String[]{});
447 }
448
449 return user;
450 }
451
452 @Override
453 public Service getService() {
454 return this;
455 }
456
457 private static class SecureBulkLoadListener implements BulkLoadListener {
458
459 private FileSystem fs;
460 private String stagingDir;
461 private Configuration conf;
462
463 private FileSystem srcFs = null;
464 private Map<String, FsPermission> origPermissions = null;
465
466 public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) {
467 this.fs = fs;
468 this.stagingDir = stagingDir;
469 this.conf = conf;
470 this.origPermissions = new HashMap<String, FsPermission>();
471 }
472
473 @Override
474 public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
475 Path p = new Path(srcPath);
476 Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
477
478
479 if (p.equals(stageP)) {
480 LOG.debug(p.getName()
481 + " is already available in staging directory. Skipping copy or rename.");
482 return stageP.toString();
483 }
484
485 if (srcFs == null) {
486 srcFs = FileSystem.newInstance(p.toUri(), conf);
487 }
488
489 if(!isFile(p)) {
490 throw new IOException("Path does not reference a file: " + p);
491 }
492
493
494 if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
495 LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " +
496 "the destination filesystem. Copying file over to destination staging dir.");
497 FileUtil.copy(srcFs, p, fs, stageP, false, conf);
498 } else {
499 LOG.debug("Moving " + p + " to " + stageP);
500 FileStatus origFileStatus = fs.getFileStatus(p);
501 origPermissions.put(srcPath, origFileStatus.getPermission());
502 if(!fs.rename(p, stageP)) {
503 throw new IOException("Failed to move HFile: " + p + " to " + stageP);
504 }
505 }
506 fs.setPermission(stageP, PERM_ALL_ACCESS);
507 return stageP.toString();
508 }
509
510 @Override
511 public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
512 LOG.debug("Bulk Load done for: " + srcPath);
513 closeSrcFs();
514 }
515
516 private void closeSrcFs() throws IOException {
517 if (srcFs != null) {
518 srcFs.close();
519 srcFs = null;
520 }
521 }
522
523 @Override
524 public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
525 try {
526 Path p = new Path(srcPath);
527 if (srcFs == null) {
528 srcFs = FileSystem.newInstance(p.toUri(), conf);
529 }
530 if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
531
532 return;
533 }
534 Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
535
536
537
538 if (p.equals(stageP)) {
539 LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
540 return;
541 }
542
543 LOG.debug("Moving " + stageP + " back to " + p);
544 if (!fs.rename(stageP, p))
545 throw new IOException("Failed to move HFile: " + stageP + " to " + p);
546
547
548 if (origPermissions.containsKey(srcPath)) {
549 fs.setPermission(p, origPermissions.get(srcPath));
550 } else {
551 LOG.warn("Can't find previous permission for path=" + srcPath);
552 }
553 } finally {
554 closeSrcFs();
555 }
556 }
557
558
559
560
561
562
563
564
565 private boolean isFile(Path p) throws IOException {
566 FileStatus status = srcFs.getFileStatus(p);
567 boolean isFile = !status.isDirectory();
568 try {
569 isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
570 } catch (Exception e) {
571 }
572 return isFile;
573 }
574 }
575 }