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