View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.security.access;
20  
21  import com.google.protobuf.RpcCallback;
22  import com.google.protobuf.RpcController;
23  import com.google.protobuf.Service;
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.FileStatus;
29  import org.apache.hadoop.fs.FileSystem;
30  import org.apache.hadoop.fs.FileUtil;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.fs.permission.FsPermission;
33  import org.apache.hadoop.hbase.Coprocessor;
34  import org.apache.hadoop.hbase.CoprocessorEnvironment;
35  import org.apache.hadoop.hbase.TableName;
36  import org.apache.hadoop.hbase.DoNotRetryIOException;
37  import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver;
38  import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
39  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
40  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
41  import org.apache.hadoop.hbase.ipc.RequestContext;
42  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
43  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
44  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
45  import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService;
46  import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
47  import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadResponse;
48  import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest;
49  import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse;
50  import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest;
51  import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
52  import org.apache.hadoop.hbase.regionserver.HRegion;
53  import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
54  import org.apache.hadoop.hbase.security.User;
55  import org.apache.hadoop.hbase.security.UserProvider;
56  import org.apache.hadoop.hbase.security.token.FsDelegationToken;
57  import org.apache.hadoop.hbase.util.Bytes;
58  import org.apache.hadoop.hbase.util.FSHDFSUtils;
59  import org.apache.hadoop.hbase.util.Methods;
60  import org.apache.hadoop.hbase.util.Pair;
61  import org.apache.hadoop.io.Text;
62  import org.apache.hadoop.security.UserGroupInformation;
63  import org.apache.hadoop.security.token.Token;
64  
65  import java.io.IOException;
66  import java.math.BigInteger;
67  import java.security.PrivilegedAction;
68  import java.security.SecureRandom;
69  import java.util.ArrayList;
70  import java.util.HashMap;
71  import java.util.List;
72  import java.util.Map;
73  
74  /**
75   * Coprocessor service for bulk loads in secure mode.
76   * This coprocessor has to be installed as part of enabling
77   * security in HBase.
78   *
79   * This service addresses two issues:
80   * <ol>
81   * <li>Moving files in a secure filesystem wherein the HBase Client
82   * and HBase Server are different filesystem users.</li>
83   * <li>Does moving in a secure manner. Assuming that the filesystem
84   * is POSIX compliant.</li>
85   * </ol>
86   *
87   * The algorithm is as follows:
88   * <ol>
89   * <li>Create an hbase owned staging directory which is
90   * world traversable (711): {@code /hbase/staging}</li>
91   * <li>A user writes out data to his secure output directory: {@code /user/foo/data}</li>
92   * <li>A call is made to hbase to create a secret staging directory
93   * which globally rwx (777): {@code /user/staging/averylongandrandomdirectoryname}</li>
94   * <li>The user moves the data into the random staging directory,
95   * then calls bulkLoadHFiles()</li>
96   * </ol>
97   * Like delegation tokens the strength of the security lies in the length
98   * and randomness of the secret directory.
99   *
100  */
101 @InterfaceAudience.Private
102 public class SecureBulkLoadEndpoint extends SecureBulkLoadService
103     implements CoprocessorService, Coprocessor {
104 
105   public static final long VERSION = 0L;
106 
107   //320/5 = 64 characters
108   private static final int RANDOM_WIDTH = 320;
109   private static final int RANDOM_RADIX = 32;
110 
111   private static Log LOG = LogFactory.getLog(SecureBulkLoadEndpoint.class);
112 
113   private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
114   private final static FsPermission PERM_HIDDEN = FsPermission.valueOf("-rwx--x--x");
115 
116   private SecureRandom random;
117   private FileSystem fs;
118   private Configuration conf;
119 
120   //two levels so it doesn't get deleted accidentally
121   //no sticky bit in Hadoop 1.0
122   private Path baseStagingDir;
123 
124   private RegionCoprocessorEnvironment env;
125 
126   private UserProvider userProvider;
127 
128   @Override
129   public void start(CoprocessorEnvironment env) {
130     this.env = (RegionCoprocessorEnvironment)env;
131     random = new SecureRandom();
132     conf = env.getConfiguration();
133     baseStagingDir = SecureBulkLoadUtil.getBaseStagingDir(conf);
134     this.userProvider = UserProvider.instantiate(conf);
135 
136     try {
137       fs = FileSystem.get(conf);
138       fs.mkdirs(baseStagingDir, PERM_HIDDEN);
139       fs.setPermission(baseStagingDir, PERM_HIDDEN);
140       //no sticky bit in hadoop-1.0, making directory nonempty so it never gets erased
141       fs.mkdirs(new Path(baseStagingDir,"DONOTERASE"), PERM_HIDDEN);
142       FileStatus status = fs.getFileStatus(baseStagingDir);
143       if(status == null) {
144         throw new IllegalStateException("Failed to create staging directory");
145       }
146       if(!status.getPermission().equals(PERM_HIDDEN)) {
147         throw new IllegalStateException(
148             "Directory already exists but permissions aren't set to '-rwx--x--x' ");
149       }
150     } catch (IOException e) {
151       throw new IllegalStateException("Failed to get FileSystem instance",e);
152     }
153   }
154 
155   @Override
156   public void stop(CoprocessorEnvironment env) throws IOException {
157   }
158 
159   @Override
160   public void prepareBulkLoad(RpcController controller,
161                                                  PrepareBulkLoadRequest request,
162                                                  RpcCallback<PrepareBulkLoadResponse> done){
163     try {
164       List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
165 
166       if(bulkLoadObservers != null) {
167         ObserverContext<RegionCoprocessorEnvironment> ctx =
168                                            new ObserverContext<RegionCoprocessorEnvironment>();
169         ctx.prepare(env);
170 
171         for(BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
172           bulkLoadObserver.prePrepareBulkLoad(ctx, request);
173         }
174       }
175 
176       String bulkToken = createStagingDir(baseStagingDir,
177           getActiveUser(), ProtobufUtil.toTableName(request.getTableName())).toString();
178       done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build());
179     } catch (IOException e) {
180       ResponseConverter.setControllerException(controller, e);
181     }
182     done.run(null);
183   }
184 
185   @Override
186   public void cleanupBulkLoad(RpcController controller,
187                               CleanupBulkLoadRequest request,
188                               RpcCallback<CleanupBulkLoadResponse> done) {
189     try {
190       List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
191 
192       if(bulkLoadObservers != null) {
193         ObserverContext<RegionCoprocessorEnvironment> ctx =
194                                            new ObserverContext<RegionCoprocessorEnvironment>();
195         ctx.prepare(env);
196 
197         for(BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
198           bulkLoadObserver.preCleanupBulkLoad(ctx, request);
199         }
200       }
201 
202       fs.delete(new Path(request.getBulkToken()), true);
203       done.run(CleanupBulkLoadResponse.newBuilder().build());
204     } catch (IOException e) {
205       ResponseConverter.setControllerException(controller, e);
206     }
207     done.run(null);
208   }
209 
210   @Override
211   public void secureBulkLoadHFiles(RpcController controller,
212                                    SecureBulkLoadHFilesRequest request,
213                                    RpcCallback<SecureBulkLoadHFilesResponse> done) {
214     final List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
215     for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
216       familyPaths.add(new Pair(el.getFamily().toByteArray(),el.getPath()));
217     }
218     
219     Token userToken = null;
220     if (userProvider.isHadoopSecurityEnabled()) {
221       userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
222               .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
223               request.getFsToken().getService()));
224     }
225     final String bulkToken = request.getBulkToken();
226     User user = getActiveUser();
227     final UserGroupInformation ugi = user.getUGI();
228     if(userToken != null) {
229       ugi.addToken(userToken);
230     } else if (userProvider.isHadoopSecurityEnabled()) {
231       //we allow this to pass through in "simple" security mode
232       //for mini cluster testing
233       ResponseConverter.setControllerException(controller,
234           new DoNotRetryIOException("User token cannot be null"));
235       done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
236       return;
237     }
238 
239     HRegion region = env.getRegion();
240     boolean bypass = false;
241     if (region.getCoprocessorHost() != null) {
242       try {
243         bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
244       } catch (IOException e) {
245         ResponseConverter.setControllerException(controller, e);
246         done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
247         return;
248       }
249     }
250     boolean loaded = false;
251     if (!bypass) {
252       // Get the target fs (HBase region server fs) delegation token
253       // Since we have checked the permission via 'preBulkLoadHFile', now let's give
254       // the 'request user' necessary token to operate on the target fs.
255       // After this point the 'doAs' user will hold two tokens, one for the source fs
256       // ('request user'), another for the target fs (HBase region server principal).
257       if (userProvider.isHadoopSecurityEnabled()) {
258         FsDelegationToken targetfsDelegationToken = new FsDelegationToken(userProvider, "renewer");
259         try {
260           targetfsDelegationToken.acquireDelegationToken(fs);
261         } catch (IOException e) {
262           ResponseConverter.setControllerException(controller, e);
263           done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
264           return;
265         }
266         Token<?> targetFsToken = targetfsDelegationToken.getUserToken();
267         if (targetFsToken != null
268             && (userToken == null || !targetFsToken.getService().equals(userToken.getService()))) {
269           ugi.addToken(targetFsToken);
270         }
271       }
272 
273       loaded = ugi.doAs(new PrivilegedAction<Boolean>() {
274         @Override
275         public Boolean run() {
276           FileSystem fs = null;
277           try {
278             Configuration conf = env.getConfiguration();
279             fs = FileSystem.get(conf);
280             for(Pair<byte[], String> el: familyPaths) {
281               Path p = new Path(el.getSecond());
282               Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
283               if(!fs.exists(stageFamily)) {
284                 fs.mkdirs(stageFamily);
285                 fs.setPermission(stageFamily, PERM_ALL_ACCESS);
286               }
287             }
288             //We call bulkLoadHFiles as requesting user
289             //To enable access prior to staging
290             return env.getRegion().bulkLoadHFiles(familyPaths, true,
291                 new SecureBulkLoadListener(fs, bulkToken, conf));
292           } catch (Exception e) {
293             LOG.error("Failed to complete bulk load", e);
294           }
295           return false;
296         }
297       });
298     }
299     if (region.getCoprocessorHost() != null) {
300       try {
301         loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
302       } catch (IOException e) {
303         ResponseConverter.setControllerException(controller, e);
304         done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
305         return;
306       }
307     }
308     done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build());
309   }
310 
311   private List<BulkLoadObserver> getBulkLoadObservers() {
312     List<BulkLoadObserver> coprocessorList =
313               this.env.getRegion().getCoprocessorHost().findCoprocessors(BulkLoadObserver.class);
314 
315     return coprocessorList;
316   }
317 
318   private Path createStagingDir(Path baseDir,
319                                 User user,
320                                 TableName tableName) throws IOException {
321     String tblName = tableName.getNameAsString().replace(":", "_");
322     String randomDir = user.getShortName()+"__"+ tblName +"__"+
323         (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
324     return createStagingDir(baseDir, user, randomDir);
325   }
326 
327   private Path createStagingDir(Path baseDir,
328                                 User user,
329                                 String randomDir) throws IOException {
330     Path p = new Path(baseDir, randomDir);
331     fs.mkdirs(p, PERM_ALL_ACCESS);
332     fs.setPermission(p, PERM_ALL_ACCESS);
333     return p;
334   }
335 
336   private User getActiveUser() {
337     User user = RequestContext.getRequestUser();
338     if (!RequestContext.isInRequestContext()) {
339       return null;
340     }
341 
342     //this is for testing
343     if (userProvider.isHadoopSecurityEnabled()
344         && "simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
345       return User.createUserForTesting(conf, user.getShortName(), new String[]{});
346     }
347 
348     return user;
349   }
350 
351   @Override
352   public Service getService() {
353     return this;
354   }
355 
356   private static class SecureBulkLoadListener implements HRegion.BulkLoadListener {
357     // Target filesystem
358     private FileSystem fs;
359     private String stagingDir;
360     private Configuration conf;
361     // Source filesystem
362     private FileSystem srcFs = null;
363     private Map<String, FsPermission> origPermissions = null;
364 
365     public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) {
366       this.fs = fs;
367       this.stagingDir = stagingDir;
368       this.conf = conf;
369       this.origPermissions = new HashMap<String, FsPermission>();
370     }
371 
372     @Override
373     public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
374       Path p = new Path(srcPath);
375       Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
376       if (srcFs == null) {
377         srcFs = FileSystem.get(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 (!FSHDFSUtils.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 {
390         LOG.debug("Moving " + p + " to " + stageP);
391         FileStatus origFileStatus = fs.getFileStatus(p);
392         origPermissions.put(srcPath, origFileStatus.getPermission());
393         if(!fs.rename(p, stageP)) {
394           throw new IOException("Failed to move HFile: " + p + " to " + stageP);
395         }
396       }
397       fs.setPermission(stageP, PERM_ALL_ACCESS);
398       return stageP.toString();
399     }
400 
401     @Override
402     public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
403       LOG.debug("Bulk Load done for: " + srcPath);
404     }
405 
406     @Override
407     public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
408       if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
409         // files are copied so no need to move them back
410         return;
411       }
412       Path p = new Path(srcPath);
413       Path stageP = new Path(stagingDir,
414           new Path(Bytes.toString(family), p.getName()));
415       LOG.debug("Moving " + stageP + " back to " + p);
416       if(!fs.rename(stageP, p))
417         throw new IOException("Failed to move HFile: " + stageP + " to " + p);
418 
419       // restore original permission
420       if (origPermissions.containsKey(srcPath)) {
421         fs.setPermission(p, origPermissions.get(srcPath));
422       } else {
423         LOG.warn("Can't find previous permission for path=" + srcPath);
424       }
425     }
426 
427     /**
428      * Check if the path is referencing a file.
429      * This is mainly needed to avoid symlinks.
430      * @param p
431      * @return true if the p is a file
432      * @throws IOException
433      */
434     private boolean isFile(Path p) throws IOException {
435       FileStatus status = srcFs.getFileStatus(p);
436       boolean isFile = !status.isDirectory();
437       try {
438         isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
439       } catch (Exception e) {
440       }
441       return isFile;
442     }
443   }
444 }