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