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    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    // if we haven't seen this ugi before, make a new counter
186    ugiReferenceCounter.compute(ugi, (key, value) -> {
187      if (value == null) {
188        value = new MutableInt(1);
189      } else {
190        value.increment();
191      }
192      return value;
193    });
194  }
195
196  private void decrementUgiReference(UserGroupInformation ugi) {
197    // if the count drops below 1 we remove the entry by returning null
198    ugiReferenceCounter.computeIfPresent(ugi, (key, value) -> {
199      if (value.intValue() > 1) {
200        value.decrement();
201      } else {
202        value = null;
203      }
204      return value;
205    });
206  }
207
208  private boolean isUserReferenced(UserGroupInformation ugi) {
209    // if the ugi is in the map, based on invariants above
210    // the count must be above zero
211    return ugiReferenceCounter.containsKey(ugi);
212  }
213
214  public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region,
215      final BulkLoadHFileRequest request) throws IOException {
216    final List<Pair<byte[], String>> familyPaths = new ArrayList<>(request.getFamilyPathCount());
217    for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
218      familyPaths.add(new Pair<>(el.getFamily().toByteArray(), el.getPath()));
219    }
220
221    Token userToken = null;
222    if (userProvider.isHadoopSecurityEnabled()) {
223      userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
224              .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
225              request.getFsToken().getService()));
226    }
227    final String bulkToken = request.getBulkToken();
228    User user = getActiveUser();
229    final UserGroupInformation ugi = user.getUGI();
230    if (userProvider.isHadoopSecurityEnabled()) {
231      try {
232        Token tok = TokenUtil.obtainToken(conn);
233        if (tok != null) {
234          boolean b = ugi.addToken(tok);
235          LOG.debug("token added " + tok + " for user " + ugi + " return=" + b);
236        }
237      } catch (IOException ioe) {
238        LOG.warn("unable to add token", ioe);
239      }
240    }
241    if (userToken != null) {
242      ugi.addToken(userToken);
243    } else if (userProvider.isHadoopSecurityEnabled()) {
244      //we allow this to pass through in "simple" security mode
245      //for mini cluster testing
246      throw new DoNotRetryIOException("User token cannot be null");
247    }
248
249    if (region.getCoprocessorHost() != null) {
250      region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
251    }
252    Map<byte[], List<Path>> map = null;
253
254    try {
255      incrementUgiReference(ugi);
256      // Get the target fs (HBase region server fs) delegation token
257      // Since we have checked the permission via 'preBulkLoadHFile', now let's give
258      // the 'request user' necessary token to operate on the target fs.
259      // After this point the 'doAs' user will hold two tokens, one for the source fs
260      // ('request user'), another for the target fs (HBase region server principal).
261      if (userProvider.isHadoopSecurityEnabled()) {
262        FsDelegationToken targetfsDelegationToken = new FsDelegationToken(userProvider,"renewer");
263        targetfsDelegationToken.acquireDelegationToken(fs);
264
265        Token<?> targetFsToken = targetfsDelegationToken.getUserToken();
266        if (targetFsToken != null
267            && (userToken == null || !targetFsToken.getService().equals(userToken.getService()))){
268          ugi.addToken(targetFsToken);
269        }
270      }
271
272      map = ugi.doAs(new PrivilegedAction<Map<byte[], List<Path>>>() {
273        @Override
274        public Map<byte[], List<Path>> run() {
275          FileSystem fs = null;
276          try {
277            fs = FileSystem.get(conf);
278            for(Pair<byte[], String> el: familyPaths) {
279              Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
280              if(!fs.exists(stageFamily)) {
281                fs.mkdirs(stageFamily);
282                fs.setPermission(stageFamily, PERM_ALL_ACCESS);
283              }
284            }
285            if (fsCreatedListener != null) {
286              fsCreatedListener.accept(region);
287            }
288            //We call bulkLoadHFiles as requesting user
289            //To enable access prior to staging
290            return region.bulkLoadHFiles(familyPaths, true,
291                new SecureBulkLoadListener(fs, bulkToken, conf), request.getCopyFile());
292          } catch (Exception e) {
293            LOG.error("Failed to complete bulk load", e);
294          }
295          return null;
296        }
297      });
298    } finally {
299      decrementUgiReference(ugi);
300      if (region.getCoprocessorHost() != null) {
301        region.getCoprocessorHost().postBulkLoadHFile(familyPaths, map);
302      }
303    }
304    return map;
305  }
306
307  private Path createStagingDir(Path baseDir,
308                                User user,
309                                TableName tableName) throws IOException {
310    String tblName = tableName.getNameAsString().replace(":", "_");
311    String randomDir = user.getShortName()+"__"+ tblName +"__"+
312        (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
313    return createStagingDir(baseDir, user, randomDir);
314  }
315
316  private Path createStagingDir(Path baseDir,
317                                User user,
318                                String randomDir) throws IOException {
319    Path p = new Path(baseDir, randomDir);
320    fs.mkdirs(p, PERM_ALL_ACCESS);
321    fs.setPermission(p, PERM_ALL_ACCESS);
322    return p;
323  }
324
325  private User getActiveUser() throws IOException {
326    // for non-rpc handling, fallback to system user
327    User user = RpcServer.getRequestUser().orElse(userProvider.getCurrent());
328    // this is for testing
329    if (userProvider.isHadoopSecurityEnabled() &&
330        "simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
331      return User.createUserForTesting(conf, user.getShortName(), new String[] {});
332    }
333
334    return user;
335  }
336
337  private static class SecureBulkLoadListener implements BulkLoadListener {
338    // Target filesystem
339    private final FileSystem fs;
340    private final String stagingDir;
341    private final Configuration conf;
342    // Source filesystem
343    private FileSystem srcFs = null;
344    private Map<String, FsPermission> origPermissions = null;
345
346    public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) {
347      this.fs = fs;
348      this.stagingDir = stagingDir;
349      this.conf = conf;
350      this.origPermissions = new HashMap<>();
351    }
352
353    @Override
354    public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile)
355        throws IOException {
356      Path p = new Path(srcPath);
357      Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
358
359      // In case of Replication for bulk load files, hfiles are already copied in staging directory
360      if (p.equals(stageP)) {
361        LOG.debug(p.getName()
362            + " is already available in staging directory. Skipping copy or rename.");
363        return stageP.toString();
364      }
365
366      if (srcFs == null) {
367        srcFs = FileSystem.newInstance(p.toUri(), conf);
368      }
369
370      if(!isFile(p)) {
371        throw new IOException("Path does not reference a file: " + p);
372      }
373
374      // Check to see if the source and target filesystems are the same
375      if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
376        LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " +
377            "the destination filesystem. Copying file over to destination staging dir.");
378        FileUtil.copy(srcFs, p, fs, stageP, false, conf);
379      } else if (copyFile) {
380        LOG.debug("Bulk-load file " + srcPath + " is copied to destination staging dir.");
381        FileUtil.copy(srcFs, p, fs, stageP, false, conf);
382      } else {
383        LOG.debug("Moving " + p + " to " + stageP);
384        FileStatus origFileStatus = fs.getFileStatus(p);
385        origPermissions.put(srcPath, origFileStatus.getPermission());
386        if(!fs.rename(p, stageP)) {
387          throw new IOException("Failed to move HFile: " + p + " to " + stageP);
388        }
389      }
390      fs.setPermission(stageP, PERM_ALL_ACCESS);
391      return stageP.toString();
392    }
393
394    @Override
395    public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
396      LOG.debug("Bulk Load done for: " + srcPath);
397      closeSrcFs();
398    }
399
400    private void closeSrcFs() throws IOException {
401      if (srcFs != null) {
402        srcFs.close();
403        srcFs = null;
404      }
405    }
406
407    @Override
408    public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
409      try {
410        Path p = new Path(srcPath);
411        if (srcFs == null) {
412          srcFs = FileSystem.newInstance(p.toUri(), conf);
413        }
414        if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
415          // files are copied so no need to move them back
416          return;
417        }
418        Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
419
420        // In case of Replication for bulk load files, hfiles are not renamed by end point during
421        // prepare stage, so no need of rename here again
422        if (p.equals(stageP)) {
423          LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
424          return;
425        }
426
427        LOG.debug("Moving " + stageP + " back to " + p);
428        if (!fs.rename(stageP, p)) {
429          throw new IOException("Failed to move HFile: " + stageP + " to " + p);
430        }
431
432        // restore original permission
433        if (origPermissions.containsKey(srcPath)) {
434          fs.setPermission(p, origPermissions.get(srcPath));
435        } else {
436          LOG.warn("Can't find previous permission for path=" + srcPath);
437        }
438      } finally {
439        closeSrcFs();
440      }
441    }
442
443    /**
444     * Check if the path is referencing a file.
445     * This is mainly needed to avoid symlinks.
446     * @param p
447     * @return true if the p is a file
448     * @throws IOException
449     */
450    private boolean isFile(Path p) throws IOException {
451      FileStatus status = srcFs.getFileStatus(p);
452      boolean isFile = !status.isDirectory();
453      try {
454        isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
455      } catch (Exception e) {
456      }
457      return isFile;
458    }
459  }
460}