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.BiFunction;
030import java.util.function.Consumer;
031
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, Integer> 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    try {
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.info("Cleaned up " + path + " successfully.");
164    } finally {
165      UserGroupInformation ugi = getActiveUser().getUGI();
166      try {
167        if (!UserGroupInformation.getLoginUser().equals(ugi) && !isUserReferenced(ugi)) {
168          FileSystem.closeAllForUGI(ugi);
169        }
170      } catch (IOException e) {
171        LOG.error("Failed to close FileSystem for: " + ugi, e);
172      }
173    }
174  }
175
176  private Consumer<HRegion> fsCreatedListener;
177
178  @VisibleForTesting
179  void setFsCreatedListener(Consumer<HRegion> fsCreatedListener) {
180    this.fsCreatedListener = fsCreatedListener;
181  }
182
183
184  private void incrementUgiReference(UserGroupInformation ugi) {
185    ugiReferenceCounter.merge(ugi, 1, new BiFunction<Integer, Integer, Integer>() {
186      @Override
187      public Integer apply(Integer oldvalue, Integer value) {
188        return ++oldvalue;
189      }
190    });
191  }
192
193  private void decrementUgiReference(UserGroupInformation ugi) {
194    ugiReferenceCounter.computeIfPresent(ugi,
195        new BiFunction<UserGroupInformation, Integer, Integer>() {
196          @Override
197          public Integer apply(UserGroupInformation key, Integer value) {
198            return value > 1 ? --value : null;
199          }
200      });
201  }
202
203  private boolean isUserReferenced(UserGroupInformation ugi) {
204    Integer count = ugiReferenceCounter.get(ugi);
205    return count != null && count > 0;
206  }
207
208  public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region,
209      final BulkLoadHFileRequest request) 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            fs = FileSystem.get(conf);
272            for(Pair<byte[], String> el: familyPaths) {
273              Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
274              if(!fs.exists(stageFamily)) {
275                fs.mkdirs(stageFamily);
276                fs.setPermission(stageFamily, PERM_ALL_ACCESS);
277              }
278            }
279            if (fsCreatedListener != null) {
280              fsCreatedListener.accept(region);
281            }
282            //We call bulkLoadHFiles as requesting user
283            //To enable access prior to staging
284            return region.bulkLoadHFiles(familyPaths, true,
285                new SecureBulkLoadListener(fs, bulkToken, conf), request.getCopyFile());
286          } catch (Exception e) {
287            LOG.error("Failed to complete bulk load", e);
288          }
289          return null;
290        }
291      });
292    } finally {
293      decrementUgiReference(ugi);
294      if (region.getCoprocessorHost() != null) {
295        region.getCoprocessorHost().postBulkLoadHFile(familyPaths, map);
296      }
297    }
298    return map;
299  }
300
301  private Path createStagingDir(Path baseDir,
302                                User user,
303                                TableName tableName) throws IOException {
304    String tblName = tableName.getNameAsString().replace(":", "_");
305    String randomDir = user.getShortName()+"__"+ tblName +"__"+
306        (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
307    return createStagingDir(baseDir, user, randomDir);
308  }
309
310  private Path createStagingDir(Path baseDir,
311                                User user,
312                                String randomDir) throws IOException {
313    Path p = new Path(baseDir, randomDir);
314    fs.mkdirs(p, PERM_ALL_ACCESS);
315    fs.setPermission(p, PERM_ALL_ACCESS);
316    return p;
317  }
318
319  private User getActiveUser() throws IOException {
320    // for non-rpc handling, fallback to system user
321    User user = RpcServer.getRequestUser().orElse(userProvider.getCurrent());
322    // this is for testing
323    if (userProvider.isHadoopSecurityEnabled() &&
324        "simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
325      return User.createUserForTesting(conf, user.getShortName(), new String[] {});
326    }
327
328    return user;
329  }
330
331  private static class SecureBulkLoadListener implements BulkLoadListener {
332    // Target filesystem
333    private final FileSystem fs;
334    private final String stagingDir;
335    private final Configuration conf;
336    // Source filesystem
337    private FileSystem srcFs = null;
338    private Map<String, FsPermission> origPermissions = null;
339
340    public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) {
341      this.fs = fs;
342      this.stagingDir = stagingDir;
343      this.conf = conf;
344      this.origPermissions = new HashMap<>();
345    }
346
347    @Override
348    public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile)
349        throws IOException {
350      Path p = new Path(srcPath);
351      Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
352
353      // In case of Replication for bulk load files, hfiles are already copied in staging directory
354      if (p.equals(stageP)) {
355        LOG.debug(p.getName()
356            + " is already available in staging directory. Skipping copy or rename.");
357        return stageP.toString();
358      }
359
360      if (srcFs == null) {
361        srcFs = FileSystem.newInstance(p.toUri(), conf);
362      }
363
364      if(!isFile(p)) {
365        throw new IOException("Path does not reference a file: " + p);
366      }
367
368      // Check to see if the source and target filesystems are the same
369      if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
370        LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " +
371            "the destination filesystem. Copying file over to destination staging dir.");
372        FileUtil.copy(srcFs, p, fs, stageP, false, conf);
373      } else if (copyFile) {
374        LOG.debug("Bulk-load file " + srcPath + " is copied to destination staging dir.");
375        FileUtil.copy(srcFs, p, fs, stageP, false, conf);
376      } else {
377        LOG.debug("Moving " + p + " to " + stageP);
378        FileStatus origFileStatus = fs.getFileStatus(p);
379        origPermissions.put(srcPath, origFileStatus.getPermission());
380        if(!fs.rename(p, stageP)) {
381          throw new IOException("Failed to move HFile: " + p + " to " + stageP);
382        }
383      }
384      fs.setPermission(stageP, PERM_ALL_ACCESS);
385      return stageP.toString();
386    }
387
388    @Override
389    public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
390      LOG.debug("Bulk Load done for: " + srcPath);
391      closeSrcFs();
392    }
393
394    private void closeSrcFs() throws IOException {
395      if (srcFs != null) {
396        srcFs.close();
397        srcFs = null;
398      }
399    }
400
401    @Override
402    public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
403      try {
404        Path p = new Path(srcPath);
405        if (srcFs == null) {
406          srcFs = FileSystem.newInstance(p.toUri(), conf);
407        }
408        if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
409          // files are copied so no need to move them back
410          return;
411        }
412        Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
413
414        // In case of Replication for bulk load files, hfiles are not renamed by end point during
415        // prepare stage, so no need of rename here again
416        if (p.equals(stageP)) {
417          LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
418          return;
419        }
420
421        LOG.debug("Moving " + stageP + " back to " + p);
422        if (!fs.rename(stageP, p)) {
423          throw new IOException("Failed to move HFile: " + stageP + " to " + p);
424        }
425
426        // restore original permission
427        if (origPermissions.containsKey(srcPath)) {
428          fs.setPermission(p, origPermissions.get(srcPath));
429        } else {
430          LOG.warn("Can't find previous permission for path=" + srcPath);
431        }
432      } finally {
433        closeSrcFs();
434      }
435    }
436
437    /**
438     * Check if the path is referencing a file.
439     * This is mainly needed to avoid symlinks.
440     * @param p
441     * @return true if the p is a file
442     * @throws IOException
443     */
444    private boolean isFile(Path p) throws IOException {
445      FileStatus status = srcFs.getFileStatus(p);
446      boolean isFile = !status.isDirectory();
447      try {
448        isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
449      } catch (Exception e) {
450      }
451      return isFile;
452    }
453  }
454}