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   *
83   * 1. Moving files in a secure filesystem wherein the HBase Client
84   * and HBase Server are different filesystem users.
85   * 2. Does moving in a secure manner. Assuming that the filesystem
86   * is POSIX compliant.
87   *
88   * The algorithm is as follows:
89   *
90   * 1. Create an hbase owned staging directory which is
91   * world traversable (711): /hbase/staging
92   * 2. A user writes out data to his secure output directory: /user/foo/data
93   * 3. A call is made to hbase to create a secret staging directory
94   * which globally rwx (777): /user/staging/averylongandrandomdirectoryname
95   * 4. The user moves the data into the random staging directory,
96   * then calls bulkLoadHFiles()
97   *
98   * Like delegation tokens the strength of the security lies in the length
99   * and randomness of the secret directory.
100  *
101  */
102 @InterfaceAudience.Private
103 public class SecureBulkLoadEndpoint extends SecureBulkLoadService
104     implements CoprocessorService, Coprocessor {
105 
106   public static final long VERSION = 0L;
107 
108   //320/5 = 64 characters
109   private static final int RANDOM_WIDTH = 320;
110   private static final int RANDOM_RADIX = 32;
111 
112   private static Log LOG = LogFactory.getLog(SecureBulkLoadEndpoint.class);
113 
114   private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
115   private final static FsPermission PERM_HIDDEN = FsPermission.valueOf("-rwx--x--x");
116 
117   private SecureRandom random;
118   private FileSystem fs;
119   private Configuration conf;
120 
121   //two levels so it doesn't get deleted accidentally
122   //no sticky bit in Hadoop 1.0
123   private Path baseStagingDir;
124 
125   private RegionCoprocessorEnvironment env;
126 
127   private UserProvider userProvider;
128 
129   @Override
130   public void start(CoprocessorEnvironment env) {
131     this.env = (RegionCoprocessorEnvironment)env;
132     random = new SecureRandom();
133     conf = env.getConfiguration();
134     baseStagingDir = SecureBulkLoadUtil.getBaseStagingDir(conf);
135     this.userProvider = UserProvider.instantiate(conf);
136 
137     try {
138       fs = FileSystem.get(conf);
139       fs.mkdirs(baseStagingDir, PERM_HIDDEN);
140       fs.setPermission(baseStagingDir, PERM_HIDDEN);
141       //no sticky bit in hadoop-1.0, making directory nonempty so it never gets erased
142       fs.mkdirs(new Path(baseStagingDir,"DONOTERASE"), PERM_HIDDEN);
143       FileStatus status = fs.getFileStatus(baseStagingDir);
144       if(status == null) {
145         throw new IllegalStateException("Failed to create staging directory");
146       }
147       if(!status.getPermission().equals(PERM_HIDDEN)) {
148         throw new IllegalStateException(
149             "Directory already exists but permissions aren't set to '-rwx--x--x' ");
150       }
151     } catch (IOException e) {
152       throw new IllegalStateException("Failed to get FileSystem instance",e);
153     }
154   }
155 
156   @Override
157   public void stop(CoprocessorEnvironment env) throws IOException {
158   }
159 
160   @Override
161   public void prepareBulkLoad(RpcController controller,
162                                                  PrepareBulkLoadRequest request,
163                                                  RpcCallback<PrepareBulkLoadResponse> done){
164     try {
165       List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
166 
167       if(bulkLoadObservers != null) {
168         ObserverContext<RegionCoprocessorEnvironment> ctx =
169                                            new ObserverContext<RegionCoprocessorEnvironment>();
170         ctx.prepare(env);
171 
172         for(BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
173           bulkLoadObserver.prePrepareBulkLoad(ctx, request);
174         }
175       }
176 
177       String bulkToken = createStagingDir(baseStagingDir,
178           getActiveUser(), ProtobufUtil.toTableName(request.getTableName())).toString();
179       done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build());
180     } catch (IOException e) {
181       ResponseConverter.setControllerException(controller, e);
182     }
183     done.run(null);
184   }
185 
186   @Override
187   public void cleanupBulkLoad(RpcController controller,
188                               CleanupBulkLoadRequest request,
189                               RpcCallback<CleanupBulkLoadResponse> done) {
190     try {
191       List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
192 
193       if(bulkLoadObservers != null) {
194         ObserverContext<RegionCoprocessorEnvironment> ctx =
195                                            new ObserverContext<RegionCoprocessorEnvironment>();
196         ctx.prepare(env);
197 
198         for(BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
199           bulkLoadObserver.preCleanupBulkLoad(ctx, request);
200         }
201       }
202 
203       fs.delete(new Path(request.getBulkToken()), 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     Region 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               Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
284               if(!fs.exists(stageFamily)) {
285                 fs.mkdirs(stageFamily);
286                 fs.setPermission(stageFamily, PERM_ALL_ACCESS);
287               }
288             }
289             //We call bulkLoadHFiles as requesting user
290             //To enable access prior to staging
291             return env.getRegion().bulkLoadHFiles(familyPaths, true,
292                 new SecureBulkLoadListener(fs, bulkToken, conf));
293           } catch (Exception e) {
294             LOG.error("Failed to complete bulk load", e);
295           }
296           return false;
297         }
298       });
299     }
300     if (region.getCoprocessorHost() != null) {
301       try {
302         loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
303       } catch (IOException e) {
304         ResponseConverter.setControllerException(controller, e);
305         done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
306         return;
307       }
308     }
309     done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build());
310   }
311 
312   private List<BulkLoadObserver> getBulkLoadObservers() {
313     List<BulkLoadObserver> coprocessorList =
314               this.env.getRegion().getCoprocessorHost().findCoprocessors(BulkLoadObserver.class);
315 
316     return coprocessorList;
317   }
318 
319   private Path createStagingDir(Path baseDir,
320                                 User user,
321                                 TableName tableName) throws IOException {
322     String tblName = tableName.getNameAsString().replace(":", "_");
323     String randomDir = user.getShortName()+"__"+ tblName +"__"+
324         (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
325     return createStagingDir(baseDir, user, randomDir);
326   }
327 
328   private Path createStagingDir(Path baseDir,
329                                 User user,
330                                 String randomDir) throws IOException {
331     Path p = new Path(baseDir, randomDir);
332     fs.mkdirs(p, PERM_ALL_ACCESS);
333     fs.setPermission(p, PERM_ALL_ACCESS);
334     return p;
335   }
336 
337   private User getActiveUser() {
338     User user = RpcServer.getRequestUser();
339     if (user == null) {
340       return null;
341     }
342 
343     //this is for testing
344     if (userProvider.isHadoopSecurityEnabled()
345         && "simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
346       return User.createUserForTesting(conf, user.getShortName(), new String[]{});
347     }
348 
349     return user;
350   }
351 
352   @Override
353   public Service getService() {
354     return this;
355   }
356 
357   private static class SecureBulkLoadListener implements BulkLoadListener {
358     // Target filesystem
359     private FileSystem fs;
360     private String stagingDir;
361     private Configuration conf;
362     // Source filesystem
363     private FileSystem srcFs = null;
364     private Map<String, FsPermission> origPermissions = null;
365 
366     public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) {
367       this.fs = fs;
368       this.stagingDir = stagingDir;
369       this.conf = conf;
370       this.origPermissions = new HashMap<String, FsPermission>();
371     }
372 
373     @Override
374     public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
375       Path p = new Path(srcPath);
376       Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
377       if (srcFs == null) {
378         srcFs = FileSystem.get(p.toUri(), conf);
379       }
380 
381       if(!isFile(p)) {
382         throw new IOException("Path does not reference a file: " + p);
383       }
384 
385       // Check to see if the source and target filesystems are the same
386       if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
387         LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " +
388             "the destination filesystem. Copying file over to destination staging dir.");
389         FileUtil.copy(srcFs, p, fs, stageP, false, conf);
390       } else {
391         LOG.debug("Moving " + p + " to " + stageP);
392         FileStatus origFileStatus = fs.getFileStatus(p);
393         origPermissions.put(srcPath, origFileStatus.getPermission());
394         if(!fs.rename(p, stageP)) {
395           throw new IOException("Failed to move HFile: " + p + " to " + stageP);
396         }
397       }
398       fs.setPermission(stageP, PERM_ALL_ACCESS);
399       return stageP.toString();
400     }
401 
402     @Override
403     public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
404       LOG.debug("Bulk Load done for: " + srcPath);
405     }
406 
407     @Override
408     public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
409       if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
410         // files are copied so no need to move them back
411         return;
412       }
413       Path p = new Path(srcPath);
414       Path stageP = new Path(stagingDir,
415           new Path(Bytes.toString(family), p.getName()));
416       LOG.debug("Moving " + stageP + " back to " + p);
417       if(!fs.rename(stageP, p))
418         throw new IOException("Failed to move HFile: " + stageP + " to " + p);
419 
420       // restore original permission
421       if (origPermissions.containsKey(srcPath)) {
422         fs.setPermission(p, origPermissions.get(srcPath));
423       } else {
424         LOG.warn("Can't find previous permission for path=" + srcPath);
425       }
426     }
427 
428     /**
429      * Check if the path is referencing a file.
430      * This is mainly needed to avoid symlinks.
431      * @param p
432      * @return true if the p is a file
433      * @throws IOException
434      */
435     private boolean isFile(Path p) throws IOException {
436       FileStatus status = srcFs.getFileStatus(p);
437       boolean isFile = !status.isDirectory();
438       try {
439         isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
440       } catch (Exception e) {
441       }
442       return isFile;
443     }
444   }
445 }