001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver; 019 020import java.io.IOException; 021import java.math.BigInteger; 022import java.security.PrivilegedAction; 023import java.security.SecureRandom; 024import java.util.ArrayList; 025import java.util.HashMap; 026import java.util.List; 027import java.util.Map; 028import java.util.concurrent.ConcurrentHashMap; 029import java.util.function.Consumer; 030import org.apache.commons.lang3.StringUtils; 031import org.apache.commons.lang3.mutable.MutableInt; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.fs.FileStatus; 034import org.apache.hadoop.fs.FileSystem; 035import org.apache.hadoop.fs.FileUtil; 036import org.apache.hadoop.fs.Path; 037import org.apache.hadoop.fs.permission.FsPermission; 038import org.apache.hadoop.hbase.DoNotRetryIOException; 039import org.apache.hadoop.hbase.HConstants; 040import org.apache.hadoop.hbase.TableName; 041import org.apache.hadoop.hbase.client.Connection; 042import org.apache.hadoop.hbase.ipc.RpcServer; 043import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener; 044import org.apache.hadoop.hbase.security.User; 045import org.apache.hadoop.hbase.security.UserProvider; 046import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier; 047import org.apache.hadoop.hbase.security.token.ClientTokenUtil; 048import org.apache.hadoop.hbase.security.token.FsDelegationToken; 049import org.apache.hadoop.hbase.util.Bytes; 050import org.apache.hadoop.hbase.util.CommonFSUtils; 051import org.apache.hadoop.hbase.util.FSUtils; 052import org.apache.hadoop.hbase.util.Methods; 053import org.apache.hadoop.hbase.util.Pair; 054import org.apache.hadoop.io.Text; 055import org.apache.hadoop.security.UserGroupInformation; 056import org.apache.hadoop.security.token.Token; 057import org.apache.yetus.audience.InterfaceAudience; 058import org.slf4j.Logger; 059import org.slf4j.LoggerFactory; 060 061import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; 062import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; 063import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest; 064import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest; 065 066/** 067 * Bulk loads in secure mode. This service addresses two issues: 068 * <ol> 069 * <li>Moving files in a secure filesystem wherein the HBase Client and HBase Server are different 070 * filesystem users.</li> 071 * <li>Does moving in a secure manner. Assuming that the filesystem is POSIX compliant.</li> 072 * </ol> 073 * The algorithm is as follows: 074 * <ol> 075 * <li>Create an hbase owned staging directory which is world traversable (711): 076 * {@code /hbase/staging}</li> 077 * <li>A user writes out data to his secure output directory: {@code /user/foo/data}</li> 078 * <li>A call is made to hbase to create a secret staging directory which globally rwx (777): 079 * {@code /user/staging/averylongandrandomdirectoryname}</li> 080 * <li>The user moves the data into the random staging directory, then calls bulkLoadHFiles()</li> 081 * </ol> 082 * Like delegation tokens the strength of the security lies in the length and randomness of the 083 * secret directory. 084 */ 085@InterfaceAudience.Private 086public class SecureBulkLoadManager { 087 088 public static final long VERSION = 0L; 089 090 // 320/5 = 64 characters 091 private static final int RANDOM_WIDTH = 320; 092 private static final int RANDOM_RADIX = 32; 093 094 private static final Logger LOG = LoggerFactory.getLogger(SecureBulkLoadManager.class); 095 096 private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx"); 097 private final static FsPermission PERM_HIDDEN = FsPermission.valueOf("-rwx--x--x"); 098 private SecureRandom random; 099 private FileSystem fs; 100 private Configuration conf; 101 102 // two levels so it doesn't get deleted accidentally 103 // no sticky bit in Hadoop 1.0 104 private Path baseStagingDir; 105 106 private UserProvider userProvider; 107 private ConcurrentHashMap<UserGroupInformation, MutableInt> ugiReferenceCounter; 108 private Connection conn; 109 110 SecureBulkLoadManager(Configuration conf, Connection conn) { 111 this.conf = conf; 112 this.conn = conn; 113 } 114 115 public void start() throws IOException { 116 random = new SecureRandom(); 117 userProvider = UserProvider.instantiate(conf); 118 ugiReferenceCounter = new ConcurrentHashMap<>(); 119 fs = FileSystem.get(conf); 120 baseStagingDir = new Path(CommonFSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME); 121 122 if (conf.get("hbase.bulkload.staging.dir") != null) { 123 LOG.warn("hbase.bulkload.staging.dir " + " is deprecated. Bulkload staging directory is " 124 + baseStagingDir); 125 } 126 if (!fs.exists(baseStagingDir)) { 127 fs.mkdirs(baseStagingDir, PERM_HIDDEN); 128 if (!PERM_HIDDEN.equals(PERM_HIDDEN.applyUMask(FsPermission.getUMask(conf)))) { 129 LOG.info("Modifying permissions to " + PERM_HIDDEN); 130 fs.setPermission(baseStagingDir, PERM_HIDDEN); 131 } 132 } 133 } 134 135 public void stop() throws IOException { 136 } 137 138 public String prepareBulkLoad(final HRegion region, final PrepareBulkLoadRequest request) 139 throws IOException { 140 User user = getActiveUser(); 141 region.getCoprocessorHost().prePrepareBulkLoad(user); 142 143 String bulkToken = 144 createStagingDir(baseStagingDir, user, region.getTableDescriptor().getTableName()).toString(); 145 146 return bulkToken; 147 } 148 149 public void cleanupBulkLoad(final HRegion region, final CleanupBulkLoadRequest request) 150 throws IOException { 151 region.getCoprocessorHost().preCleanupBulkLoad(getActiveUser()); 152 153 Path path = new Path(request.getBulkToken()); 154 if (!fs.delete(path, true)) { 155 if (fs.exists(path)) { 156 throw new IOException("Failed to clean up " + path); 157 } 158 } 159 LOG.trace("Cleaned up {} successfully.", path); 160 } 161 162 private Consumer<HRegion> fsCreatedListener; 163 164 void setFsCreatedListener(Consumer<HRegion> fsCreatedListener) { 165 this.fsCreatedListener = fsCreatedListener; 166 } 167 168 private void incrementUgiReference(UserGroupInformation ugi) { 169 // if we haven't seen this ugi before, make a new counter 170 ugiReferenceCounter.compute(ugi, (key, value) -> { 171 if (value == null) { 172 value = new MutableInt(1); 173 } else { 174 value.increment(); 175 } 176 return value; 177 }); 178 } 179 180 private void decrementUgiReference(UserGroupInformation ugi) { 181 // if the count drops below 1 we remove the entry by returning null 182 ugiReferenceCounter.computeIfPresent(ugi, (key, value) -> { 183 if (value.intValue() > 1) { 184 value.decrement(); 185 } else { 186 value = null; 187 } 188 return value; 189 }); 190 } 191 192 private boolean isUserReferenced(UserGroupInformation ugi) { 193 // if the ugi is in the map, based on invariants above 194 // the count must be above zero 195 return ugiReferenceCounter.containsKey(ugi); 196 } 197 198 public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region, 199 final BulkLoadHFileRequest request) throws IOException { 200 return secureBulkLoadHFiles(region, request, null); 201 } 202 203 public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region, 204 final BulkLoadHFileRequest request, List<String> clusterIds) throws IOException { 205 final List<Pair<byte[], String>> familyPaths = new ArrayList<>(request.getFamilyPathCount()); 206 for (ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) { 207 familyPaths.add(new Pair<>(el.getFamily().toByteArray(), el.getPath())); 208 } 209 210 Token userToken = null; 211 if (userProvider.isHadoopSecurityEnabled()) { 212 userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), 213 request.getFsToken().getPassword().toByteArray(), new Text(request.getFsToken().getKind()), 214 new Text(request.getFsToken().getService())); 215 } 216 final String bulkToken = request.getBulkToken(); 217 User user = getActiveUser(); 218 final UserGroupInformation ugi = user.getUGI(); 219 if (userProvider.isHadoopSecurityEnabled()) { 220 try { 221 Token<AuthenticationTokenIdentifier> tok = ClientTokenUtil.obtainToken(conn); 222 if (tok != null) { 223 boolean b = ugi.addToken(tok); 224 LOG.debug("token added " + tok + " for user " + ugi + " return=" + b); 225 } 226 } catch (IOException ioe) { 227 LOG.warn("unable to add token", ioe); 228 } 229 } 230 if (userToken != null) { 231 ugi.addToken(userToken); 232 } else if (userProvider.isHadoopSecurityEnabled()) { 233 // we allow this to pass through in "simple" security mode 234 // for mini cluster testing 235 throw new DoNotRetryIOException("User token cannot be null"); 236 } 237 238 if (region.getCoprocessorHost() != null) { 239 region.getCoprocessorHost().preBulkLoadHFile(familyPaths); 240 } 241 Map<byte[], List<Path>> map = null; 242 243 try { 244 incrementUgiReference(ugi); 245 // Get the target fs (HBase region server fs) delegation token 246 // Since we have checked the permission via 'preBulkLoadHFile', now let's give 247 // the 'request user' necessary token to operate on the target fs. 248 // After this point the 'doAs' user will hold two tokens, one for the source fs 249 // ('request user'), another for the target fs (HBase region server principal). 250 if (userProvider.isHadoopSecurityEnabled()) { 251 FsDelegationToken targetfsDelegationToken = new FsDelegationToken(userProvider, "renewer"); 252 targetfsDelegationToken.acquireDelegationToken(fs); 253 254 Token<?> targetFsToken = targetfsDelegationToken.getUserToken(); 255 if ( 256 targetFsToken != null 257 && (userToken == null || !targetFsToken.getService().equals(userToken.getService())) 258 ) { 259 ugi.addToken(targetFsToken); 260 } 261 } 262 263 map = ugi.doAs(new PrivilegedAction<Map<byte[], List<Path>>>() { 264 @Override 265 public Map<byte[], List<Path>> run() { 266 FileSystem fs = null; 267 try { 268 /* 269 * This is creating and caching a new FileSystem instance. Other code called "beneath" 270 * this method will rely on this FileSystem instance being in the cache. This is 271 * important as those methods make _no_ attempt to close this FileSystem instance. It is 272 * critical that here, in SecureBulkLoadManager, we are tracking the lifecycle and 273 * closing the FS when safe to do so. 274 */ 275 fs = FileSystem.get(conf); 276 for (Pair<byte[], String> el : familyPaths) { 277 Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst())); 278 if (!fs.exists(stageFamily)) { 279 fs.mkdirs(stageFamily); 280 fs.setPermission(stageFamily, PERM_ALL_ACCESS); 281 } 282 } 283 if (fsCreatedListener != null) { 284 fsCreatedListener.accept(region); 285 } 286 // We call bulkLoadHFiles as requesting user 287 // To enable access prior to staging 288 return region.bulkLoadHFiles(familyPaths, true, 289 new SecureBulkLoadListener(fs, bulkToken, conf), request.getCopyFile(), clusterIds, 290 request.getReplicate()); 291 } catch (Exception e) { 292 LOG.error("Failed to complete bulk load", e); 293 } 294 return null; 295 } 296 }); 297 } finally { 298 decrementUgiReference(ugi); 299 try { 300 if (!UserGroupInformation.getLoginUser().equals(ugi) && !isUserReferenced(ugi)) { 301 FileSystem.closeAllForUGI(ugi); 302 } 303 } catch (IOException e) { 304 LOG.error("Failed to close FileSystem for: {}", ugi, e); 305 } 306 if (region.getCoprocessorHost() != null) { 307 region.getCoprocessorHost().postBulkLoadHFile(familyPaths, map); 308 } 309 } 310 return map; 311 } 312 313 private Path createStagingDir(Path baseDir, User user, TableName tableName) throws IOException { 314 String tblName = tableName.getNameAsString().replace(":", "_"); 315 String randomDir = user.getShortName() + "__" + tblName + "__" 316 + (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX)); 317 return createStagingDir(baseDir, user, randomDir); 318 } 319 320 private Path createStagingDir(Path baseDir, User user, String randomDir) throws IOException { 321 Path p = new Path(baseDir, randomDir); 322 fs.mkdirs(p, PERM_ALL_ACCESS); 323 fs.setPermission(p, PERM_ALL_ACCESS); 324 return p; 325 } 326 327 private User getActiveUser() throws IOException { 328 // for non-rpc handling, fallback to system user 329 User user = RpcServer.getRequestUser().orElse(userProvider.getCurrent()); 330 // this is for testing 331 if ( 332 userProvider.isHadoopSecurityEnabled() 333 && "simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY)) 334 ) { 335 return User.createUserForTesting(conf, user.getShortName(), new String[] {}); 336 } 337 338 return user; 339 } 340 341 // package-private for test purpose only 342 static class SecureBulkLoadListener implements BulkLoadListener { 343 // Target filesystem 344 private final FileSystem fs; 345 private final String stagingDir; 346 private final Configuration conf; 347 // Source filesystem 348 private FileSystem srcFs = null; 349 private Map<String, FsPermission> origPermissions = null; 350 private Map<String, String> origSources = null; 351 352 public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) { 353 this.fs = fs; 354 this.stagingDir = stagingDir; 355 this.conf = conf; 356 this.origPermissions = new HashMap<>(); 357 this.origSources = new HashMap<>(); 358 } 359 360 @Override 361 public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile, 362 String customStaging) throws IOException { 363 Path p = new Path(srcPath); 364 365 // store customStaging for failedBulkLoad 366 String currentStaging = stagingDir; 367 if (StringUtils.isNotEmpty(customStaging)) { 368 currentStaging = customStaging; 369 } 370 371 Path stageP = new Path(currentStaging, new Path(Bytes.toString(family), p.getName())); 372 373 // In case of Replication for bulk load files, hfiles are already copied in staging directory 374 if (p.equals(stageP)) { 375 LOG.debug( 376 p.getName() + " is already available in staging directory. Skipping copy or rename."); 377 return stageP.toString(); 378 } 379 380 if (srcFs == null) { 381 srcFs = FileSystem.newInstance(p.toUri(), conf); 382 } 383 384 if (!isFile(p)) { 385 throw new IOException("Path does not reference a file: " + p); 386 } 387 388 // Check to see if the source and target filesystems are the same 389 if (!FSUtils.isSameHdfs(conf, srcFs, fs)) { 390 LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " 391 + "the destination filesystem. Copying file over to destination staging dir."); 392 FileUtil.copy(srcFs, p, fs, stageP, false, conf); 393 } else if (copyFile) { 394 LOG.debug("Bulk-load file " + srcPath + " is copied to destination staging dir."); 395 FileUtil.copy(srcFs, p, fs, stageP, false, conf); 396 } else { 397 LOG.debug("Moving " + p + " to " + stageP); 398 FileStatus origFileStatus = fs.getFileStatus(p); 399 origPermissions.put(srcPath, origFileStatus.getPermission()); 400 origSources.put(stageP.toString(), srcPath); 401 if (!fs.rename(p, stageP)) { 402 throw new IOException("Failed to move HFile: " + p + " to " + stageP); 403 } 404 } 405 fs.setPermission(stageP, PERM_ALL_ACCESS); 406 407 return stageP.toString(); 408 } 409 410 @Override 411 public void doneBulkLoad(byte[] family, String srcPath) throws IOException { 412 LOG.debug("Bulk Load done for: " + srcPath); 413 closeSrcFs(); 414 } 415 416 private void closeSrcFs() throws IOException { 417 if (srcFs != null) { 418 srcFs.close(); 419 srcFs = null; 420 } 421 } 422 423 @Override 424 public void failedBulkLoad(final byte[] family, final String stagedPath) throws IOException { 425 try { 426 String src = origSources.get(stagedPath); 427 if (StringUtils.isEmpty(src)) { 428 LOG.debug(stagedPath + " was not moved to staging. No need to move back"); 429 return; 430 } 431 432 Path stageP = new Path(stagedPath); 433 if (!fs.exists(stageP)) { 434 throw new IOException( 435 "Missing HFile: " + stageP + ", can't be moved back to it's original place"); 436 } 437 438 // we should not move back files if the original exists 439 Path srcPath = new Path(src); 440 if (srcFs.exists(srcPath)) { 441 LOG.debug(src + " is already at it's original place. No need to move."); 442 return; 443 } 444 445 LOG.debug("Moving " + stageP + " back to " + srcPath); 446 if (!fs.rename(stageP, srcPath)) { 447 throw new IOException("Failed to move HFile: " + stageP + " to " + srcPath); 448 } 449 450 // restore original permission 451 if (origPermissions.containsKey(stagedPath)) { 452 fs.setPermission(srcPath, origPermissions.get(src)); 453 } else { 454 LOG.warn("Can't find previous permission for path=" + stagedPath); 455 } 456 } finally { 457 closeSrcFs(); 458 } 459 } 460 461 /** 462 * Check if the path is referencing a file. This is mainly needed to avoid symlinks. 463 * @return true if the p is a file 464 */ 465 private boolean isFile(Path p) throws IOException { 466 FileStatus status = srcFs.getFileStatus(p); 467 boolean isFile = !status.isDirectory(); 468 try { 469 isFile = 470 isFile && !(Boolean) Methods.call(FileStatus.class, status, "isSymlink", null, null); 471 } catch (Exception e) { 472 } 473 return isFile; 474 } 475 } 476}