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