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
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
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
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
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
122
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
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
233
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
254
255
256
257
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
290
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
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
359 private FileSystem fs;
360 private String stagingDir;
361 private Configuration conf;
362
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
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
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
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
430
431
432
433
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 }