1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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.HashMap;
71 import java.util.List;
72 import java.util.Map;
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100 @InterfaceAudience.Private
101 public class SecureBulkLoadEndpoint extends SecureBulkLoadService
102 implements CoprocessorService, Coprocessor {
103
104 public static final long VERSION = 0L;
105
106
107 private static final int RANDOM_WIDTH = 320;
108 private static final int RANDOM_RADIX = 32;
109
110 private static Log LOG = LogFactory.getLog(SecureBulkLoadEndpoint.class);
111
112 private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
113 private final static FsPermission PERM_HIDDEN = FsPermission.valueOf("-rwx--x--x");
114
115 private SecureRandom random;
116 private FileSystem fs;
117 private Configuration conf;
118
119
120
121 private Path baseStagingDir;
122
123 private RegionCoprocessorEnvironment env;
124
125 private UserProvider userProvider;
126
127 @Override
128 public void start(CoprocessorEnvironment env) {
129 this.env = (RegionCoprocessorEnvironment)env;
130 random = new SecureRandom();
131 conf = env.getConfiguration();
132 baseStagingDir = SecureBulkLoadUtil.getBaseStagingDir(conf);
133 this.userProvider = UserProvider.instantiate(conf);
134
135 try {
136 fs = FileSystem.get(conf);
137 fs.mkdirs(baseStagingDir, PERM_HIDDEN);
138 fs.setPermission(baseStagingDir, PERM_HIDDEN);
139
140 fs.mkdirs(new Path(baseStagingDir,"DONOTERASE"), PERM_HIDDEN);
141 FileStatus status = fs.getFileStatus(baseStagingDir);
142 if(status == null) {
143 throw new IllegalStateException("Failed to create staging directory");
144 }
145 if(!status.getPermission().equals(PERM_HIDDEN)) {
146 throw new IllegalStateException(
147 "Directory already exists but permissions aren't set to '-rwx--x--x' ");
148 }
149 } catch (IOException e) {
150 throw new IllegalStateException("Failed to get FileSystem instance",e);
151 }
152 }
153
154 @Override
155 public void stop(CoprocessorEnvironment env) throws IOException {
156 }
157
158 @Override
159 public void prepareBulkLoad(RpcController controller,
160 PrepareBulkLoadRequest request,
161 RpcCallback<PrepareBulkLoadResponse> done){
162 try {
163 List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
164
165 if(bulkLoadObservers != null) {
166 ObserverContext<RegionCoprocessorEnvironment> ctx =
167 new ObserverContext<RegionCoprocessorEnvironment>();
168 ctx.prepare(env);
169
170 for(BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
171 bulkLoadObserver.prePrepareBulkLoad(ctx, request);
172 }
173 }
174
175 String bulkToken = createStagingDir(baseStagingDir,
176 getActiveUser(), ProtobufUtil.toTableName(request.getTableName())).toString();
177 done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build());
178 } catch (IOException e) {
179 ResponseConverter.setControllerException(controller, e);
180 }
181 done.run(null);
182 }
183
184 @Override
185 public void cleanupBulkLoad(RpcController controller,
186 CleanupBulkLoadRequest request,
187 RpcCallback<CleanupBulkLoadResponse> done) {
188 try {
189 List<BulkLoadObserver> bulkLoadObservers = getBulkLoadObservers();
190
191 if(bulkLoadObservers != null) {
192 ObserverContext<RegionCoprocessorEnvironment> ctx =
193 new ObserverContext<RegionCoprocessorEnvironment>();
194 ctx.prepare(env);
195
196 for(BulkLoadObserver bulkLoadObserver : bulkLoadObservers) {
197 bulkLoadObserver.preCleanupBulkLoad(ctx, request);
198 }
199 }
200
201 fs.delete(new Path(request.getBulkToken()), true);
202 done.run(CleanupBulkLoadResponse.newBuilder().build());
203 } catch (IOException e) {
204 ResponseConverter.setControllerException(controller, e);
205 }
206 done.run(null);
207 }
208
209 @Override
210 public void secureBulkLoadHFiles(RpcController controller,
211 SecureBulkLoadHFilesRequest request,
212 RpcCallback<SecureBulkLoadHFilesResponse> done) {
213 final List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
214 for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
215 familyPaths.add(new Pair(el.getFamily().toByteArray(),el.getPath()));
216 }
217
218 Token userToken = null;
219 if (userProvider.isHadoopSecurityEnabled()) {
220 userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
221 .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
222 request.getFsToken().getService()));
223 }
224 final String bulkToken = request.getBulkToken();
225 User user = getActiveUser();
226 final UserGroupInformation ugi = user.getUGI();
227 if(userToken != null) {
228 ugi.addToken(userToken);
229 } else if (userProvider.isHadoopSecurityEnabled()) {
230
231
232 ResponseConverter.setControllerException(controller,
233 new DoNotRetryIOException("User token cannot be null"));
234 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
235 return;
236 }
237
238 HRegion region = env.getRegion();
239 boolean bypass = false;
240 if (region.getCoprocessorHost() != null) {
241 try {
242 bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
243 } catch (IOException e) {
244 ResponseConverter.setControllerException(controller, e);
245 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
246 return;
247 }
248 }
249 boolean loaded = false;
250 if (!bypass) {
251
252
253
254
255
256 if (userProvider.isHadoopSecurityEnabled()) {
257 FsDelegationToken targetfsDelegationToken = new FsDelegationToken(userProvider, "renewer");
258 try {
259 targetfsDelegationToken.acquireDelegationToken(fs);
260 } catch (IOException e) {
261 ResponseConverter.setControllerException(controller, e);
262 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
263 return;
264 }
265 Token<?> targetFsToken = targetfsDelegationToken.getUserToken();
266 if (targetFsToken != null
267 && (userToken == null || !targetFsToken.getService().equals(userToken.getService()))) {
268 ugi.addToken(targetFsToken);
269 }
270 }
271
272 loaded = ugi.doAs(new PrivilegedAction<Boolean>() {
273 @Override
274 public Boolean run() {
275 FileSystem fs = null;
276 try {
277 Configuration conf = env.getConfiguration();
278 fs = FileSystem.get(conf);
279 for(Pair<byte[], String> el: familyPaths) {
280 Path p = new Path(el.getSecond());
281 Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
282 if(!fs.exists(stageFamily)) {
283 fs.mkdirs(stageFamily);
284 fs.setPermission(stageFamily, PERM_ALL_ACCESS);
285 }
286 }
287
288
289 return env.getRegion().bulkLoadHFiles(familyPaths, true,
290 new SecureBulkLoadListener(fs, bulkToken, conf));
291 } catch (Exception e) {
292 LOG.error("Failed to complete bulk load", e);
293 }
294 return false;
295 }
296 });
297 }
298 if (region.getCoprocessorHost() != null) {
299 try {
300 loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
301 } catch (IOException e) {
302 ResponseConverter.setControllerException(controller, e);
303 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(false).build());
304 return;
305 }
306 }
307 done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build());
308 }
309
310 private List<BulkLoadObserver> getBulkLoadObservers() {
311 List<BulkLoadObserver> coprocessorList =
312 this.env.getRegion().getCoprocessorHost().findCoprocessors(BulkLoadObserver.class);
313
314 return coprocessorList;
315 }
316
317 private Path createStagingDir(Path baseDir,
318 User user,
319 TableName tableName) throws IOException {
320 String tblName = tableName.getNameAsString().replace(":", "_");
321 String randomDir = user.getShortName()+"__"+ tblName +"__"+
322 (new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
323 return createStagingDir(baseDir, user, randomDir);
324 }
325
326 private Path createStagingDir(Path baseDir,
327 User user,
328 String randomDir) throws IOException {
329 Path p = new Path(baseDir, randomDir);
330 fs.mkdirs(p, PERM_ALL_ACCESS);
331 fs.setPermission(p, PERM_ALL_ACCESS);
332 return p;
333 }
334
335 private User getActiveUser() {
336 User user = RequestContext.getRequestUser();
337 if (!RequestContext.isInRequestContext()) {
338 return null;
339 }
340
341
342 if (userProvider.isHadoopSecurityEnabled()
343 && "simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
344 return User.createUserForTesting(conf, user.getShortName(), new String[]{});
345 }
346
347 return user;
348 }
349
350 @Override
351 public Service getService() {
352 return this;
353 }
354
355 private static class SecureBulkLoadListener implements HRegion.BulkLoadListener {
356
357 private FileSystem fs;
358 private String stagingDir;
359 private Configuration conf;
360
361 private FileSystem srcFs = null;
362 private Map<String, FsPermission> origPermissions = null;
363
364 public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) {
365 this.fs = fs;
366 this.stagingDir = stagingDir;
367 this.conf = conf;
368 this.origPermissions = new HashMap<String, FsPermission>();
369 }
370
371 @Override
372 public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
373 Path p = new Path(srcPath);
374 Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
375 if (srcFs == null) {
376 srcFs = FileSystem.get(p.toUri(), conf);
377 }
378
379 if(!isFile(p)) {
380 throw new IOException("Path does not reference a file: " + p);
381 }
382
383
384 if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
385 LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " +
386 "the destination filesystem. Copying file over to destination staging dir.");
387 FileUtil.copy(srcFs, p, fs, stageP, false, conf);
388 } else {
389 LOG.debug("Moving " + p + " to " + stageP);
390 FileStatus origFileStatus = fs.getFileStatus(p);
391 origPermissions.put(srcPath, origFileStatus.getPermission());
392 if(!fs.rename(p, stageP)) {
393 throw new IOException("Failed to move HFile: " + p + " to " + stageP);
394 }
395 }
396 fs.setPermission(stageP, PERM_ALL_ACCESS);
397 return stageP.toString();
398 }
399
400 @Override
401 public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
402 LOG.debug("Bulk Load done for: " + srcPath);
403 }
404
405 @Override
406 public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
407 if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
408
409 return;
410 }
411 Path p = new Path(srcPath);
412 Path stageP = new Path(stagingDir,
413 new Path(Bytes.toString(family), p.getName()));
414 LOG.debug("Moving " + stageP + " back to " + p);
415 if(!fs.rename(stageP, p))
416 throw new IOException("Failed to move HFile: " + stageP + " to " + p);
417
418
419 if (origPermissions.containsKey(srcPath)) {
420 fs.setPermission(p, origPermissions.get(srcPath));
421 } else {
422 LOG.warn("Can't find previous permission for path=" + srcPath);
423 }
424 }
425
426
427
428
429
430
431
432
433 private boolean isFile(Path p) throws IOException {
434 FileStatus status = srcFs.getFileStatus(p);
435 boolean isFile = !status.isDir();
436 try {
437 isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
438 } catch (Exception e) {
439 }
440 return isFile;
441 }
442 }
443 }