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