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