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