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