HBASE-5498 Secure Bulk Load (Francis Liu)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1433452 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
474b61483b
commit
7fb425daf4
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,71 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_outer_classname = "SecureBulkLoadProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
import 'hbase.proto';
|
||||
import 'Client.proto';
|
||||
|
||||
message SecureBulkLoadHFilesRequest {
|
||||
repeated BulkLoadHFileRequest.FamilyPath familyPath = 1;
|
||||
optional bool assignSeqNum = 2;
|
||||
required DelegationTokenProto fsToken = 3;
|
||||
required string bulkToken = 4;
|
||||
}
|
||||
|
||||
message SecureBulkLoadHFilesResponse {
|
||||
required bool loaded = 1;
|
||||
}
|
||||
|
||||
message DelegationTokenProto {
|
||||
optional bytes identifier = 1;
|
||||
optional bytes password = 2;
|
||||
optional string kind = 3;
|
||||
optional string service = 4;
|
||||
}
|
||||
|
||||
message PrepareBulkLoadRequest {
|
||||
required bytes tableName = 1;
|
||||
}
|
||||
|
||||
message PrepareBulkLoadResponse {
|
||||
required string bulkToken = 1;
|
||||
}
|
||||
|
||||
message CleanupBulkLoadRequest {
|
||||
required string bulkToken = 1;
|
||||
|
||||
}
|
||||
|
||||
message CleanupBulkLoadResponse {
|
||||
}
|
||||
|
||||
service SecureBulkLoadService {
|
||||
rpc prepareBulkLoad(PrepareBulkLoadRequest)
|
||||
returns (PrepareBulkLoadResponse);
|
||||
|
||||
rpc secureBulkLoadHFiles(SecureBulkLoadHFilesRequest)
|
||||
returns (SecureBulkLoadHFilesResponse);
|
||||
|
||||
rpc cleanupBulkLoad(CleanupBulkLoadRequest)
|
||||
returns (CleanupBulkLoadResponse);
|
||||
}
|
|
@ -0,0 +1,175 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.client.coprocessor;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos;
|
||||
import org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Client proxy for SecureBulkLoadProtocol
|
||||
* used in conjunction with SecureBulkLoadEndpoint
|
||||
*/
|
||||
public class SecureBulkLoadClient {
|
||||
private HTable table;
|
||||
|
||||
public SecureBulkLoadClient(HTable table) {
|
||||
this.table = table;
|
||||
}
|
||||
|
||||
public String prepareBulkLoad(final byte[] tableName) throws IOException {
|
||||
try {
|
||||
return
|
||||
table.coprocessorService(SecureBulkLoadProtos.SecureBulkLoadService.class,
|
||||
HConstants.EMPTY_START_ROW,
|
||||
HConstants.EMPTY_START_ROW,
|
||||
new Batch.Call<SecureBulkLoadProtos.SecureBulkLoadService,String>() {
|
||||
@Override
|
||||
public String call(SecureBulkLoadProtos.SecureBulkLoadService instance) throws IOException {
|
||||
ServerRpcController controller = new ServerRpcController();
|
||||
|
||||
BlockingRpcCallback<SecureBulkLoadProtos.PrepareBulkLoadResponse> rpcCallback =
|
||||
new BlockingRpcCallback<SecureBulkLoadProtos.PrepareBulkLoadResponse>();
|
||||
|
||||
SecureBulkLoadProtos.PrepareBulkLoadRequest request =
|
||||
SecureBulkLoadProtos.PrepareBulkLoadRequest.newBuilder()
|
||||
.setTableName(com.google.protobuf.ByteString.copyFrom(tableName)).build();
|
||||
|
||||
instance.prepareBulkLoad(controller,
|
||||
request,
|
||||
rpcCallback);
|
||||
|
||||
SecureBulkLoadProtos.PrepareBulkLoadResponse response = rpcCallback.get();
|
||||
if (controller.failedOnException()) {
|
||||
throw controller.getFailedOn();
|
||||
}
|
||||
return response.getBulkToken();
|
||||
}
|
||||
}).entrySet().iterator().next().getValue();
|
||||
} catch (Throwable throwable) {
|
||||
throw new IOException(throwable);
|
||||
}
|
||||
}
|
||||
|
||||
public void cleanupBulkLoad(final String bulkToken) throws IOException {
|
||||
try {
|
||||
table.coprocessorService(SecureBulkLoadProtos.SecureBulkLoadService.class,
|
||||
HConstants.EMPTY_START_ROW,
|
||||
HConstants.EMPTY_START_ROW,
|
||||
new Batch.Call<SecureBulkLoadProtos.SecureBulkLoadService, String>() {
|
||||
|
||||
@Override
|
||||
public String call(SecureBulkLoadProtos.SecureBulkLoadService instance) throws IOException {
|
||||
ServerRpcController controller = new ServerRpcController();
|
||||
|
||||
BlockingRpcCallback<SecureBulkLoadProtos.CleanupBulkLoadResponse> rpcCallback =
|
||||
new BlockingRpcCallback<SecureBulkLoadProtos.CleanupBulkLoadResponse>();
|
||||
|
||||
SecureBulkLoadProtos.CleanupBulkLoadRequest request =
|
||||
SecureBulkLoadProtos.CleanupBulkLoadRequest.newBuilder()
|
||||
.setBulkToken(bulkToken).build();
|
||||
|
||||
instance.cleanupBulkLoad(controller,
|
||||
request,
|
||||
rpcCallback);
|
||||
|
||||
if (controller.failedOnException()) {
|
||||
throw controller.getFailedOn();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
});
|
||||
} catch (Throwable throwable) {
|
||||
throw new IOException(throwable);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean bulkLoadHFiles(final List<Pair<byte[], String>> familyPaths,
|
||||
final Token<?> userToken,
|
||||
final String bulkToken,
|
||||
final byte[] startRow) throws IOException {
|
||||
try {
|
||||
return
|
||||
table.coprocessorService(SecureBulkLoadProtos.SecureBulkLoadService.class,
|
||||
startRow,
|
||||
startRow,
|
||||
new Batch.Call<SecureBulkLoadProtos.SecureBulkLoadService,Boolean>() {
|
||||
|
||||
@Override
|
||||
public Boolean call(SecureBulkLoadProtos.SecureBulkLoadService instance) throws IOException {
|
||||
SecureBulkLoadProtos.DelegationTokenProto protoDT =
|
||||
SecureBulkLoadProtos.DelegationTokenProto.newBuilder().build();
|
||||
if(userToken != null) {
|
||||
protoDT =
|
||||
SecureBulkLoadProtos.DelegationTokenProto.newBuilder()
|
||||
.setIdentifier(ByteString.copyFrom(userToken.getIdentifier()))
|
||||
.setPassword(ByteString.copyFrom(userToken.getPassword()))
|
||||
.setKind(userToken.getKind().toString())
|
||||
.setService(userToken.getService().toString()).build();
|
||||
}
|
||||
|
||||
List<ClientProtos.BulkLoadHFileRequest.FamilyPath> protoFamilyPaths =
|
||||
new ArrayList<ClientProtos.BulkLoadHFileRequest.FamilyPath>();
|
||||
for(Pair<byte[], String> el: familyPaths) {
|
||||
protoFamilyPaths.add(ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder()
|
||||
.setFamily(ByteString.copyFrom(el.getFirst()))
|
||||
.setPath(el.getSecond()).build());
|
||||
}
|
||||
|
||||
SecureBulkLoadProtos.SecureBulkLoadHFilesRequest request =
|
||||
SecureBulkLoadProtos.SecureBulkLoadHFilesRequest.newBuilder()
|
||||
.setFsToken(protoDT)
|
||||
.addAllFamilyPath(protoFamilyPaths)
|
||||
.setBulkToken(bulkToken).build();
|
||||
|
||||
ServerRpcController controller = new ServerRpcController();
|
||||
BlockingRpcCallback<SecureBulkLoadProtos.SecureBulkLoadHFilesResponse> rpcCallback =
|
||||
new BlockingRpcCallback<SecureBulkLoadProtos.SecureBulkLoadHFilesResponse>();
|
||||
instance.secureBulkLoadHFiles(controller,
|
||||
request,
|
||||
rpcCallback);
|
||||
|
||||
SecureBulkLoadProtos.SecureBulkLoadHFilesResponse response = rpcCallback.get();
|
||||
if (controller.failedOnException()) {
|
||||
throw controller.getFailedOn();
|
||||
}
|
||||
return response.getLoaded();
|
||||
}
|
||||
}).entrySet().iterator().next().getValue();
|
||||
} catch (Throwable throwable) {
|
||||
throw new IOException(throwable);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getStagingPath(String bulkToken, byte[] family) throws IOException {
|
||||
return SecureBulkLoadEndpoint.getStagingPath(table.getConfiguration(), bulkToken, family);
|
||||
}
|
||||
}
|
|
@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
|
|||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.ServerCallable;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.SecureBulkLoadClient;
|
||||
import org.apache.hadoop.hbase.io.HalfStoreFileReader;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
|
||||
|
@ -74,8 +75,10 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
|
@ -100,10 +103,21 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
private static final String ASSIGN_SEQ_IDS = "hbase.mapreduce.bulkload.assign.sequenceNumbers";
|
||||
private boolean assignSeqIds;
|
||||
|
||||
public LoadIncrementalHFiles(Configuration conf) throws Exception {
|
||||
private boolean useSecure;
|
||||
private Token<?> userToken;
|
||||
private String bulkToken;
|
||||
|
||||
//package private for testing
|
||||
LoadIncrementalHFiles(Configuration conf, Boolean useSecure) throws Exception {
|
||||
super(conf);
|
||||
this.cfg = conf;
|
||||
this.hbAdmin = new HBaseAdmin(conf);
|
||||
//added simple for testing
|
||||
this.useSecure = useSecure != null ? useSecure : User.isHBaseSecurityEnabled(conf);
|
||||
}
|
||||
|
||||
public LoadIncrementalHFiles(Configuration conf) throws Exception {
|
||||
this(conf, null);
|
||||
assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
|
||||
}
|
||||
|
||||
|
@ -215,6 +229,18 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
return;
|
||||
}
|
||||
|
||||
//If using secure bulk load
|
||||
//prepare staging directory and token
|
||||
if(useSecure) {
|
||||
FileSystem fs = FileSystem.get(cfg);
|
||||
//This condition is here for unit testing
|
||||
//Since delegation token doesn't work in mini cluster
|
||||
if(User.isSecurityEnabled()) {
|
||||
userToken = fs.getDelegationToken("renewer");
|
||||
}
|
||||
bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(table.getTableName());
|
||||
}
|
||||
|
||||
// Assumes that region splits can happen while this occurs.
|
||||
while (!queue.isEmpty()) {
|
||||
// need to reload split keys each iteration.
|
||||
|
@ -243,6 +269,18 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
}
|
||||
|
||||
} finally {
|
||||
if(useSecure) {
|
||||
if(userToken != null) {
|
||||
try {
|
||||
userToken.cancel(cfg);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Failed to cancel HDFS delegation token.", e);
|
||||
}
|
||||
}
|
||||
if(bulkToken != null) {
|
||||
new SecureBulkLoadClient(table).cleanupBulkLoad(bulkToken);
|
||||
}
|
||||
}
|
||||
pool.shutdown();
|
||||
if (queue != null && !queue.isEmpty()) {
|
||||
StringBuilder err = new StringBuilder();
|
||||
|
@ -476,11 +514,47 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
tableName, first) {
|
||||
@Override
|
||||
public Boolean call() throws Exception {
|
||||
SecureBulkLoadClient secureClient = null;
|
||||
boolean success = false;
|
||||
|
||||
try {
|
||||
LOG.debug("Going to connect to server " + location + " for row "
|
||||
+ Bytes.toStringBinary(row));
|
||||
byte[] regionName = location.getRegionInfo().getRegionName();
|
||||
return ProtobufUtil.bulkLoadHFile(server, famPaths, regionName,
|
||||
assignSeqIds);
|
||||
if(!useSecure) {
|
||||
success = ProtobufUtil.bulkLoadHFile(server, famPaths, regionName, assignSeqIds);
|
||||
} else {
|
||||
HTable table = new HTable(conn.getConfiguration(), tableName);
|
||||
secureClient = new SecureBulkLoadClient(table);
|
||||
success = secureClient.bulkLoadHFiles(famPaths, userToken, bulkToken, location.getRegionInfo().getStartKey());
|
||||
}
|
||||
return success;
|
||||
} finally {
|
||||
//Best effort copying of files that might not have been imported
|
||||
//from the staging directory back to original location
|
||||
//in user directory
|
||||
if(secureClient != null && !success) {
|
||||
FileSystem fs = FileSystem.get(cfg);
|
||||
for(Pair<byte[], String> el : famPaths) {
|
||||
Path hfileStagingPath = null;
|
||||
Path hfileOrigPath = new Path(el.getSecond());
|
||||
try {
|
||||
hfileStagingPath= new Path(secureClient.getStagingPath(bulkToken, el.getFirst()),
|
||||
hfileOrigPath.getName());
|
||||
if(fs.rename(hfileStagingPath, hfileOrigPath)) {
|
||||
LOG.debug("Moved back file " + hfileOrigPath + " from " +
|
||||
hfileStagingPath);
|
||||
} else if(fs.exists(hfileStagingPath)){
|
||||
LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
|
||||
hfileStagingPath);
|
||||
}
|
||||
} catch(Exception ex) {
|
||||
LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
|
||||
hfileStagingPath, ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -3254,17 +3254,25 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
return multipleFamilies;
|
||||
}
|
||||
|
||||
|
||||
public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
|
||||
boolean assignSeqId) throws IOException {
|
||||
return bulkLoadHFiles(familyPaths, assignSeqId, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempts to atomically load a group of hfiles. This is critical for loading
|
||||
* rows with multiple column families atomically.
|
||||
*
|
||||
* @param familyPaths List of Pair<byte[] column family, String hfilePath>
|
||||
* @param bulkLoadListener Internal hooks enabling massaging/preparation of a
|
||||
* file about to be bulk loaded
|
||||
* @param assignSeqId
|
||||
* @return true if successful, false if failed recoverably
|
||||
* @throws IOException if failed unrecoverably.
|
||||
*/
|
||||
public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
|
||||
boolean assignSeqId) throws IOException {
|
||||
public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, boolean assignSeqId,
|
||||
BulkLoadListener bulkLoadListener) throws IOException {
|
||||
Preconditions.checkNotNull(familyPaths);
|
||||
// we need writeLock for multi-family bulk load
|
||||
startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
|
||||
|
@ -3324,7 +3332,14 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
String path = p.getSecond();
|
||||
Store store = getStore(familyName);
|
||||
try {
|
||||
store.bulkLoadHFile(path, assignSeqId ? this.log.obtainSeqNum() : -1);
|
||||
String finalPath = path;
|
||||
if(bulkLoadListener != null) {
|
||||
finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
|
||||
}
|
||||
store.bulkLoadHFile(finalPath, assignSeqId ? this.log.obtainSeqNum() : -1);
|
||||
if(bulkLoadListener != null) {
|
||||
bulkLoadListener.doneBulkLoad(familyName, path);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
// A failure here can cause an atomicity violation that we currently
|
||||
// cannot recover from since it is likely a failed HDFS operation.
|
||||
|
@ -3332,6 +3347,14 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// TODO Need a better story for reverting partial failures due to HDFS.
|
||||
LOG.error("There was a partial failure due to IO when attempting to" +
|
||||
" load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond(), ioe);
|
||||
if(bulkLoadListener != null) {
|
||||
try {
|
||||
bulkLoadListener.failedBulkLoad(familyName, path);
|
||||
} catch (Exception ex) {
|
||||
LOG.error("Error while calling failedBulkLoad for family "+
|
||||
Bytes.toString(familyName)+" with path "+path, ex);
|
||||
}
|
||||
}
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
|
@ -5452,4 +5475,38 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
if (bc != null) bc.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Listener class to enable callers of
|
||||
* bulkLoadHFile() to perform any necessary
|
||||
* pre/post processing of a given bulkload call
|
||||
*/
|
||||
public static interface BulkLoadListener {
|
||||
|
||||
/**
|
||||
* Called before an HFile is actually loaded
|
||||
* @param family family being loaded to
|
||||
* @param srcPath path of HFile
|
||||
* @return final path to be used for actual loading
|
||||
* @throws IOException
|
||||
*/
|
||||
String prepareBulkLoad(byte[] family, String srcPath) throws IOException;
|
||||
|
||||
/**
|
||||
* Called after a successful HFile load
|
||||
* @param family family being loaded to
|
||||
* @param srcPath path of HFile
|
||||
* @throws IOException
|
||||
*/
|
||||
void doneBulkLoad(byte[] family, String srcPath) throws IOException;
|
||||
|
||||
/**
|
||||
* Called after a failed HFile load
|
||||
* @param family family being loaded to
|
||||
* @param srcPath path of HFile
|
||||
* @throws IOException
|
||||
*/
|
||||
void failedBulkLoad(byte[] family, String srcPath) throws IOException;
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -577,7 +577,11 @@ public class HStore implements Store, StoreConfiguration {
|
|||
// Copy the file if it's on another filesystem
|
||||
FileSystem srcFs = srcPath.getFileSystem(conf);
|
||||
FileSystem desFs = fs instanceof HFileSystem ? ((HFileSystem)fs).getBackingFs() : fs;
|
||||
if (!srcFs.equals(desFs)) {
|
||||
//We can't compare FileSystem instances as
|
||||
//equals() includes UGI instance as part of the comparison
|
||||
//and won't work when doing SecureBulkLoad
|
||||
//TODO deal with viewFS
|
||||
if (!srcFs.getUri().equals(desFs.getUri())) {
|
||||
LOG.info("Bulk-load file " + srcPath + " is on different filesystem than " +
|
||||
"the destination store. Copying file over to destination filesystem.");
|
||||
Path tmpPath = getTmpPath();
|
||||
|
|
|
@ -18,7 +18,9 @@ import java.io.IOException;
|
|||
import java.net.InetAddress;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -63,6 +65,7 @@ import org.apache.hadoop.hbase.security.User;
|
|||
import org.apache.hadoop.hbase.security.access.Permission.Action;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.ListMultimap;
|
||||
|
@ -1005,6 +1008,76 @@ public class AccessController extends BaseRegionObserver
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies user has WRITE privileges on
|
||||
* the Column Families involved in the bulkLoadHFile
|
||||
* request. Specific Column Write privileges are presently
|
||||
* ignored.
|
||||
*/
|
||||
@Override
|
||||
public void preBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||
List<Pair<byte[], String>> familyPaths) throws IOException {
|
||||
List<byte[]> cfs = new LinkedList<byte[]>();
|
||||
for(Pair<byte[],String> el : familyPaths) {
|
||||
requirePermission("preBulkLoadHFile",
|
||||
ctx.getEnvironment().getRegion().getTableDesc().getName(),
|
||||
el.getFirst(),
|
||||
null,
|
||||
Permission.Action.WRITE);
|
||||
}
|
||||
}
|
||||
|
||||
private AuthResult hasSomeAccess(RegionCoprocessorEnvironment e, String method, Action action) throws IOException {
|
||||
User requestUser = getActiveUser();
|
||||
byte[] tableName = e.getRegion().getTableDesc().getName();
|
||||
AuthResult authResult = permissionGranted(method, requestUser,
|
||||
action, e, Collections.EMPTY_MAP);
|
||||
if (!authResult.isAllowed()) {
|
||||
for(UserPermission userPerm:
|
||||
AccessControlLists.getUserPermissions(regionEnv.getConfiguration(), tableName)) {
|
||||
for(Permission.Action userAction: userPerm.getActions()) {
|
||||
if(userAction.equals(action)) {
|
||||
return AuthResult.allow(method, "Access allowed", requestUser,
|
||||
action, tableName, null, null);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return authResult;
|
||||
}
|
||||
|
||||
/**
|
||||
* Authorization check for
|
||||
* SecureBulkLoadProtocol.prepareBulkLoad()
|
||||
* @param e
|
||||
* @throws IOException
|
||||
*/
|
||||
//TODO this should end up as a coprocessor hook
|
||||
public void prePrepareBulkLoad(RegionCoprocessorEnvironment e) throws IOException {
|
||||
AuthResult authResult = hasSomeAccess(e, "prePrepareBulkLoad", Action.WRITE);
|
||||
logResult(authResult);
|
||||
if (!authResult.isAllowed()) {
|
||||
throw new AccessDeniedException("Insufficient permissions (table=" +
|
||||
e.getRegion().getTableDesc().getNameAsString() + ", action=WRITE)");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Authorization security check for
|
||||
* SecureBulkLoadProtocol.cleanupBulkLoad()
|
||||
* @param e
|
||||
* @throws IOException
|
||||
*/
|
||||
//TODO this should end up as a coprocessor hook
|
||||
public void preCleanupBulkLoad(RegionCoprocessorEnvironment e) throws IOException {
|
||||
AuthResult authResult = hasSomeAccess(e, "preCleanupBulkLoad", Action.WRITE);
|
||||
logResult(authResult);
|
||||
if (!authResult.isAllowed()) {
|
||||
throw new AccessDeniedException("Insufficient permissions (table=" +
|
||||
e.getRegion().getTableDesc().getNameAsString() + ", action=WRITE)");
|
||||
}
|
||||
}
|
||||
|
||||
/* ---- Protobuf AccessControlService implementation ---- */
|
||||
@Override
|
||||
public void grant(RpcController controller,
|
||||
|
|
|
@ -0,0 +1,368 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.security.access;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.ipc.RequestContext;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Methods;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigInteger;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.security.SecureRandom;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Coprocessor service for bulk loads in secure mode.
|
||||
* This coprocessor has to be installed as part of enabling
|
||||
* security in HBase.
|
||||
*
|
||||
* This service addresses two issues:
|
||||
*
|
||||
* 1. Moving files in a secure filesystem wherein the HBase Client
|
||||
* and HBase Server are different filesystem users.
|
||||
* 2. Does moving in a secure manner. Assuming that the filesystem
|
||||
* is POSIX compliant.
|
||||
*
|
||||
* The algorithm is as follows:
|
||||
*
|
||||
* 1. Create an hbase owned staging directory which is
|
||||
* world traversable (711): /hbase/staging
|
||||
* 2. A user writes out data to his secure output directory: /user/foo/data
|
||||
* 3. A call is made to hbase to create a secret staging directory
|
||||
* which globally rwx (777): /user/staging/averylongandrandomdirectoryname
|
||||
* 4. The user makes the data world readable and writable, then moves it
|
||||
* into the random staging directory, then calls bulkLoadHFiles()
|
||||
*
|
||||
* Like delegation tokens the strength of the security lies in the length
|
||||
* and randomness of the secret directory.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
||||
implements CoprocessorService, Coprocessor {
|
||||
|
||||
public static final long VERSION = 0L;
|
||||
|
||||
//320/5 = 64 characters
|
||||
private static final int RANDOM_WIDTH = 320;
|
||||
private static final int RANDOM_RADIX = 32;
|
||||
|
||||
private static Log LOG = LogFactory.getLog(SecureBulkLoadEndpoint.class);
|
||||
|
||||
private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
|
||||
private final static FsPermission PERM_HIDDEN = FsPermission.valueOf("-rwx--x--x");
|
||||
private final static String BULKLOAD_STAGING_DIR = "hbase.bulkload.staging.dir";
|
||||
|
||||
private SecureRandom random;
|
||||
private FileSystem fs;
|
||||
private Configuration conf;
|
||||
|
||||
//two levels so it doesn't get deleted accidentally
|
||||
//no sticky bit in Hadoop 1.0
|
||||
private Path baseStagingDir;
|
||||
|
||||
private RegionCoprocessorEnvironment env;
|
||||
|
||||
|
||||
@Override
|
||||
public void start(CoprocessorEnvironment env) {
|
||||
this.env = (RegionCoprocessorEnvironment)env;
|
||||
random = new SecureRandom();
|
||||
conf = env.getConfiguration();
|
||||
baseStagingDir = getBaseStagingDir(conf);
|
||||
|
||||
try {
|
||||
fs = FileSystem.get(conf);
|
||||
fs.mkdirs(baseStagingDir, PERM_HIDDEN);
|
||||
fs.setPermission(baseStagingDir, PERM_HIDDEN);
|
||||
//no sticky bit in hadoop-1.0, making directory nonempty so it never gets erased
|
||||
fs.mkdirs(new Path(baseStagingDir,"DONOTERASE"), PERM_HIDDEN);
|
||||
FileStatus status = fs.getFileStatus(baseStagingDir);
|
||||
if(status == null) {
|
||||
throw new IllegalStateException("Failed to create staging directory");
|
||||
}
|
||||
if(!status.getPermission().equals(PERM_HIDDEN)) {
|
||||
throw new IllegalStateException(
|
||||
"Directory already exists but permissions aren't set to '-rwx--x--x' ");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException("Failed to get FileSystem instance",e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(CoprocessorEnvironment env) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void prepareBulkLoad(RpcController controller,
|
||||
PrepareBulkLoadRequest request,
|
||||
RpcCallback<PrepareBulkLoadResponse> done){
|
||||
try {
|
||||
getAccessController().prePrepareBulkLoad(env);
|
||||
String bulkToken = createStagingDir(baseStagingDir,
|
||||
getActiveUser(), request.getTableName().toByteArray()).toString();
|
||||
done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build());
|
||||
} catch (IOException e) {
|
||||
ResponseConverter.setControllerException(controller, e);
|
||||
}
|
||||
done.run(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanupBulkLoad(RpcController controller,
|
||||
CleanupBulkLoadRequest request,
|
||||
RpcCallback<CleanupBulkLoadResponse> done) {
|
||||
try {
|
||||
getAccessController().preCleanupBulkLoad(env);
|
||||
fs.delete(createStagingDir(baseStagingDir,
|
||||
getActiveUser(),
|
||||
env.getRegion().getTableDesc().getName(),
|
||||
new Path(request.getBulkToken()).getName()),
|
||||
true);
|
||||
done.run(CleanupBulkLoadResponse.newBuilder().build());
|
||||
} catch (IOException e) {
|
||||
ResponseConverter.setControllerException(controller, e);
|
||||
}
|
||||
done.run(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void secureBulkLoadHFiles(RpcController controller,
|
||||
SecureBulkLoadHFilesRequest request,
|
||||
RpcCallback<SecureBulkLoadHFilesResponse> done) {
|
||||
final List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
|
||||
for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
|
||||
familyPaths.add(new Pair(el.getFamily().toByteArray(),el.getPath()));
|
||||
}
|
||||
final Token userToken =
|
||||
new Token(request.getFsToken().getIdentifier().toByteArray(),
|
||||
request.getFsToken().getPassword().toByteArray(),
|
||||
new Text(request.getFsToken().getKind()),
|
||||
new Text(request.getFsToken().getService()));
|
||||
final String bulkToken = request.getBulkToken();
|
||||
User user = getActiveUser();
|
||||
final UserGroupInformation ugi = user.getUGI();
|
||||
if(userToken != null) {
|
||||
ugi.addToken(userToken);
|
||||
} else if(User.isSecurityEnabled()) {
|
||||
//we allow this to pass through in "simple" security mode
|
||||
//for mini cluster testing
|
||||
ResponseConverter.setControllerException(controller,
|
||||
new DoNotRetryIOException("User token cannot be null"));
|
||||
return;
|
||||
}
|
||||
|
||||
HRegion region = env.getRegion();
|
||||
boolean bypass = false;
|
||||
if (region.getCoprocessorHost() != null) {
|
||||
try {
|
||||
bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
|
||||
} catch (IOException e) {
|
||||
ResponseConverter.setControllerException(controller, e);
|
||||
done.run(null);
|
||||
return;
|
||||
}
|
||||
}
|
||||
boolean loaded = false;
|
||||
if (!bypass) {
|
||||
loaded = ugi.doAs(new PrivilegedAction<Boolean>() {
|
||||
@Override
|
||||
public Boolean run() {
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
Configuration conf = env.getConfiguration();
|
||||
fs = FileSystem.get(conf);
|
||||
for(Pair<byte[], String> el: familyPaths) {
|
||||
Path p = new Path(el.getSecond());
|
||||
LOG.trace("Setting permission for: " + p);
|
||||
fs.setPermission(p, PERM_ALL_ACCESS);
|
||||
Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
|
||||
if(!fs.exists(stageFamily)) {
|
||||
fs.mkdirs(stageFamily);
|
||||
fs.setPermission(stageFamily, PERM_ALL_ACCESS);
|
||||
}
|
||||
}
|
||||
//We call bulkLoadHFiles as requesting user
|
||||
//To enable access prior to staging
|
||||
return env.getRegion().bulkLoadHFiles(familyPaths, true,
|
||||
new SecureBulkLoadListener(fs, bulkToken));
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to complete bulk load", e);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
});
|
||||
}
|
||||
if (region.getCoprocessorHost() != null) {
|
||||
try {
|
||||
loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
|
||||
} catch (IOException e) {
|
||||
ResponseConverter.setControllerException(controller, e);
|
||||
done.run(null);
|
||||
return;
|
||||
}
|
||||
}
|
||||
done.run(SecureBulkLoadHFilesResponse.newBuilder().setLoaded(loaded).build());
|
||||
}
|
||||
|
||||
private AccessController getAccessController() {
|
||||
return (AccessController) this.env.getRegion()
|
||||
.getCoprocessorHost().findCoprocessor(AccessController.class.getName());
|
||||
}
|
||||
|
||||
private Path createStagingDir(Path baseDir, User user, byte[] tableName) throws IOException {
|
||||
String randomDir = user.getShortName()+"__"+Bytes.toString(tableName)+"__"+
|
||||
(new BigInteger(RANDOM_WIDTH, random).toString(RANDOM_RADIX));
|
||||
return createStagingDir(baseDir, user, tableName, randomDir);
|
||||
}
|
||||
|
||||
private Path createStagingDir(Path baseDir,
|
||||
User user,
|
||||
byte[] tableName,
|
||||
String randomDir) throws IOException {
|
||||
Path p = new Path(baseDir, randomDir);
|
||||
fs.mkdirs(p, PERM_ALL_ACCESS);
|
||||
fs.setPermission(p, PERM_ALL_ACCESS);
|
||||
return p;
|
||||
}
|
||||
|
||||
private User getActiveUser() {
|
||||
User user = RequestContext.getRequestUser();
|
||||
if (!RequestContext.isInRequestContext()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
//this is for testing
|
||||
if("simple".equalsIgnoreCase(conf.get(User.HBASE_SECURITY_CONF_KEY))) {
|
||||
return User.createUserForTesting(conf, user.getShortName(), new String[]{});
|
||||
}
|
||||
|
||||
return user;
|
||||
}
|
||||
|
||||
/**
|
||||
* This returns the staging path for a given column family.
|
||||
* This is needed for clean recovery and called reflectively in LoadIncrementalHFiles
|
||||
*/
|
||||
public static Path getStagingPath(Configuration conf, String bulkToken, byte[] family) {
|
||||
Path stageP = new Path(getBaseStagingDir(conf), bulkToken);
|
||||
return new Path(stageP, Bytes.toString(family));
|
||||
}
|
||||
|
||||
private static Path getBaseStagingDir(Configuration conf) {
|
||||
return new Path(conf.get(BULKLOAD_STAGING_DIR, "/tmp/hbase-staging"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Service getService() {
|
||||
return this;
|
||||
}
|
||||
|
||||
private static class SecureBulkLoadListener implements HRegion.BulkLoadListener {
|
||||
private FileSystem fs;
|
||||
private String stagingDir;
|
||||
|
||||
public SecureBulkLoadListener(FileSystem fs, String stagingDir) {
|
||||
this.fs = fs;
|
||||
this.stagingDir = stagingDir;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
|
||||
Path p = new Path(srcPath);
|
||||
Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
|
||||
|
||||
if(!isFile(p)) {
|
||||
throw new IOException("Path does not reference a file: " + p);
|
||||
}
|
||||
|
||||
LOG.debug("Moving " + p + " to " + stageP);
|
||||
if(!fs.rename(p, stageP)) {
|
||||
throw new IOException("Failed to move HFile: " + p + " to " + stageP);
|
||||
}
|
||||
return stageP.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
|
||||
LOG.debug("Bulk Load done for: " + srcPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
|
||||
Path p = new Path(srcPath);
|
||||
Path stageP = new Path(stagingDir,
|
||||
new Path(Bytes.toString(family), p.getName()));
|
||||
LOG.debug("Moving " + stageP + " back to " + p);
|
||||
if(!fs.rename(stageP, p))
|
||||
throw new IOException("Failed to move HFile: " + stageP + " to " + p);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the path is referencing a file.
|
||||
* This is mainly needed to avoid symlinks.
|
||||
* @param p
|
||||
* @return true if the p is a file
|
||||
* @throws IOException
|
||||
*/
|
||||
private boolean isFile(Path p) throws IOException {
|
||||
FileStatus status = fs.getFileStatus(p);
|
||||
boolean isFile = !status.isDir();
|
||||
try {
|
||||
isFile = isFile && !(Boolean)Methods.call(FileStatus.class, status, "isSymlink", null, null);
|
||||
} catch (Exception e) {
|
||||
}
|
||||
return isFile;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -62,7 +62,9 @@ public class TestLoadIncrementalHFiles {
|
|||
public static String COMPRESSION =
|
||||
Compression.Algorithm.NONE.getName();
|
||||
|
||||
private static HBaseTestingUtility util = new HBaseTestingUtility();
|
||||
static HBaseTestingUtility util = new HBaseTestingUtility();
|
||||
//used by secure subclass
|
||||
static boolean useSecure = false;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
|
@ -151,8 +153,7 @@ public class TestLoadIncrementalHFiles {
|
|||
|
||||
HTable table = new HTable(util.getConfiguration(), TABLE);
|
||||
util.waitTableAvailable(TABLE, 30000);
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(
|
||||
util.getConfiguration());
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration(), useSecure);
|
||||
loader.doBulkLoad(dir, table);
|
||||
|
||||
assertEquals(expectedRows, util.countRows(table));
|
||||
|
|
|
@ -71,7 +71,9 @@ import com.google.protobuf.ServiceException;
|
|||
public class TestLoadIncrementalHFilesSplitRecovery {
|
||||
final static Log LOG = LogFactory.getLog(TestHRegionServerBulkLoad.class);
|
||||
|
||||
private static HBaseTestingUtility util;
|
||||
static HBaseTestingUtility util;
|
||||
//used by secure subclass
|
||||
static boolean useSecure = false;
|
||||
|
||||
final static int NUM_CFS = 10;
|
||||
final static byte[] QUAL = Bytes.toBytes("qual");
|
||||
|
@ -138,8 +140,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
*/
|
||||
private void populateTable(String table, int value) throws Exception {
|
||||
// create HFiles for different column families
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
|
||||
util.getConfiguration());
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration(), useSecure);
|
||||
Path bulk1 = buildBulkFiles(table, value);
|
||||
HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table));
|
||||
lih.doBulkLoad(bulk1, t);
|
||||
|
@ -231,7 +232,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
final AtomicInteger attmptedCalls = new AtomicInteger();
|
||||
final AtomicInteger failedCalls = new AtomicInteger();
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
|
||||
util.getConfiguration()) {
|
||||
util.getConfiguration(), useSecure) {
|
||||
|
||||
protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
|
||||
byte[] tableName, final byte[] first, Collection<LoadQueueItem> lqis)
|
||||
|
@ -299,7 +300,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
// files to fail when attempt to atomically import. This is recoverable.
|
||||
final AtomicInteger attemptedCalls = new AtomicInteger();
|
||||
LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles(
|
||||
util.getConfiguration()) {
|
||||
util.getConfiguration(), useSecure) {
|
||||
|
||||
protected void bulkLoadPhase(final HTable htable, final HConnection conn,
|
||||
ExecutorService pool, Deque<LoadQueueItem> queue,
|
||||
|
@ -340,7 +341,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
|
||||
final AtomicInteger countedLqis= new AtomicInteger();
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
|
||||
util.getConfiguration()) {
|
||||
util.getConfiguration(), useSecure) {
|
||||
protected List<LoadQueueItem> groupOrSplit(
|
||||
Multimap<ByteBuffer, LoadQueueItem> regionGroups,
|
||||
final LoadQueueItem item, final HTable htable,
|
||||
|
@ -372,7 +373,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
setupTable(table, 10);
|
||||
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
|
||||
util.getConfiguration()) {
|
||||
util.getConfiguration(), useSecure) {
|
||||
int i = 0;
|
||||
|
||||
protected List<LoadQueueItem> groupOrSplit(
|
||||
|
|
|
@ -0,0 +1,56 @@
|
|||
/**
|
||||
* Copyright The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import org.apache.hadoop.hbase.LargeTests;
|
||||
import org.apache.hadoop.hbase.security.access.AccessControlLists;
|
||||
import org.apache.hadoop.hbase.security.access.SecureTestUtil;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Reruns TestLoadIncrementalHFiles using LoadIncrementalHFiles in secure mode.
|
||||
* This suite is unable to verify the security handoff/turnover
|
||||
* as miniCluster is running as system user thus has root privileges
|
||||
* and delegation tokens don't seem to work on miniDFS.
|
||||
*
|
||||
* Thus SecureBulkload can only be completely verified by running
|
||||
* integration tests against a secure cluster. This suite is still
|
||||
* invaluable as it verifies the other mechanisms that need to be
|
||||
* supported as part of a LoadIncrementalFiles call.
|
||||
*/
|
||||
@Category(LargeTests.class)
|
||||
public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles{
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
useSecure = true;
|
||||
// setup configuration
|
||||
SecureTestUtil.enableSecurity(util.getConfiguration());
|
||||
|
||||
util.startMiniCluster();
|
||||
|
||||
// Wait for the ACL table to become available
|
||||
util.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,66 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.LargeTests;
|
||||
import org.apache.hadoop.hbase.security.access.AccessControlLists;
|
||||
import org.apache.hadoop.hbase.security.access.SecureTestUtil;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
||||
/**
|
||||
* Reruns TestSecureLoadIncrementalHFilesSplitRecovery
|
||||
* using LoadIncrementalHFiles in secure mode.
|
||||
* This suite is unable to verify the security handoff/turnover
|
||||
* as miniCluster is running as system user thus has root privileges
|
||||
* and delegation tokens don't seem to work on miniDFS.
|
||||
*
|
||||
* Thus SecureBulkload can only be completely verified by running
|
||||
* integration tests against a secure cluster. This suite is still
|
||||
* invaluable as it verifies the other mechanisms that need to be
|
||||
* supported as part of a LoadIncrementalFiles call.
|
||||
*/
|
||||
@Category(LargeTests.class)
|
||||
public class TestSecureLoadIncrementalHFilesSplitRecovery extends TestLoadIncrementalHFilesSplitRecovery {
|
||||
|
||||
//This "overrides" the parent static method
|
||||
//make sure they are in sync
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
useSecure = true;
|
||||
util = new HBaseTestingUtility();
|
||||
// setup configuration
|
||||
SecureTestUtil.enableSecurity(util.getConfiguration());
|
||||
|
||||
util.startMiniCluster();
|
||||
|
||||
// Wait for the ACL table to become available
|
||||
util.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
|
||||
}
|
||||
|
||||
//Disabling this test as it does not work in secure mode
|
||||
@Test
|
||||
@Override
|
||||
public void testBulkLoadPhaseFailure() {
|
||||
}
|
||||
}
|
||||
|
|
@ -31,7 +31,8 @@ public class SecureTestUtil {
|
|||
conf.set("hadoop.security.authorization", "false");
|
||||
conf.set("hadoop.security.authentication", "simple");
|
||||
conf.set("hbase.coprocessor.master.classes", AccessController.class.getName());
|
||||
conf.set("hbase.coprocessor.region.classes", AccessController.class.getName());
|
||||
conf.set("hbase.coprocessor.region.classes", AccessController.class.getName()+
|
||||
","+SecureBulkLoadEndpoint.class.getName());
|
||||
// add the process running user to superusers
|
||||
String currentUser = User.getCurrent().getName();
|
||||
conf.set("hbase.superuser", "admin,"+currentUser);
|
||||
|
|
|
@ -30,12 +30,17 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.LargeTests;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
|
@ -53,6 +58,9 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
|
@ -199,22 +207,29 @@ public class TestAccessController {
|
|||
try {
|
||||
user.runAs(action);
|
||||
fail("Expected AccessDeniedException for user '" + user.getShortName() + "'");
|
||||
} catch (RetriesExhaustedWithDetailsException e) {
|
||||
} catch (IOException e) {
|
||||
boolean isAccessDeniedException = false;
|
||||
if(e instanceof RetriesExhaustedWithDetailsException) {
|
||||
// in case of batch operations, and put, the client assembles a
|
||||
// RetriesExhaustedWithDetailsException instead of throwing an
|
||||
// AccessDeniedException
|
||||
boolean isAccessDeniedException = false;
|
||||
for (Throwable ex : e.getCauses()) {
|
||||
if (ex instanceof ServiceException) {
|
||||
ServiceException se = (ServiceException)ex;
|
||||
if (se.getCause() != null && se.getCause() instanceof AccessDeniedException) {
|
||||
for(Throwable ex : ((RetriesExhaustedWithDetailsException) e).getCauses()) {
|
||||
if (ex instanceof AccessDeniedException) {
|
||||
isAccessDeniedException = true;
|
||||
break;
|
||||
}
|
||||
} else if (ex instanceof AccessDeniedException) {
|
||||
}
|
||||
}
|
||||
else {
|
||||
// For doBulkLoad calls AccessDeniedException
|
||||
// is buried in the stack trace
|
||||
Throwable ex = e;
|
||||
do {
|
||||
if (ex instanceof AccessDeniedException) {
|
||||
isAccessDeniedException = true;
|
||||
break;
|
||||
}
|
||||
} while((ex = ex.getCause()) != null);
|
||||
}
|
||||
if (!isAccessDeniedException) {
|
||||
fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
|
||||
|
@ -233,8 +248,6 @@ public class TestAccessController {
|
|||
}
|
||||
}
|
||||
fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
|
||||
} catch (AccessDeniedException ade) {
|
||||
// expected result
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -674,6 +687,104 @@ public class TestAccessController {
|
|||
verifyReadWrite(checkAndPut);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBulkLoad() throws Exception {
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
final Path dir = TEST_UTIL.getDataTestDir("testBulkLoad");
|
||||
fs.mkdirs(dir);
|
||||
//need to make it globally writable
|
||||
//so users creating HFiles have write permissions
|
||||
fs.setPermission(dir, FsPermission.valueOf("-rwxrwxrwx"));
|
||||
|
||||
PrivilegedExceptionAction bulkLoadAction = new PrivilegedExceptionAction() {
|
||||
public Object run() throws Exception {
|
||||
int numRows = 3;
|
||||
|
||||
//Making the assumption that the test table won't split between the range
|
||||
byte[][][] hfileRanges = {{{(byte)0}, {(byte)9}}};
|
||||
|
||||
Path bulkLoadBasePath = new Path(dir, new Path(User.getCurrent().getName()));
|
||||
new BulkLoadHelper(bulkLoadBasePath)
|
||||
.bulkLoadHFile(TEST_TABLE, TEST_FAMILY, Bytes.toBytes("q"), hfileRanges, numRows);
|
||||
|
||||
return null;
|
||||
}
|
||||
};
|
||||
verifyWrite(bulkLoadAction);
|
||||
}
|
||||
|
||||
public class BulkLoadHelper {
|
||||
private final FileSystem fs;
|
||||
private final Path loadPath;
|
||||
private final Configuration conf;
|
||||
|
||||
public BulkLoadHelper(Path loadPath) throws IOException {
|
||||
fs = TEST_UTIL.getTestFileSystem();
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
loadPath = loadPath.makeQualified(fs);
|
||||
this.loadPath = loadPath;
|
||||
}
|
||||
|
||||
private void createHFile(Path path,
|
||||
byte[] family, byte[] qualifier,
|
||||
byte[] startKey, byte[] endKey, int numRows) throws IOException {
|
||||
|
||||
HFile.Writer writer = null;
|
||||
long now = System.currentTimeMillis();
|
||||
try {
|
||||
writer = HFile.getWriterFactory(conf, new CacheConfig(conf))
|
||||
.withPath(fs, path)
|
||||
.withComparator(KeyValue.KEY_COMPARATOR)
|
||||
.create();
|
||||
// subtract 2 since numRows doesn't include boundary keys
|
||||
for (byte[] key : Bytes.iterateOnSplits(startKey, endKey, true, numRows-2)) {
|
||||
KeyValue kv = new KeyValue(key, family, qualifier, now, key);
|
||||
writer.append(kv);
|
||||
}
|
||||
} finally {
|
||||
if(writer != null)
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void bulkLoadHFile(
|
||||
byte[] tableName,
|
||||
byte[] family,
|
||||
byte[] qualifier,
|
||||
byte[][][] hfileRanges,
|
||||
int numRowsPerRange) throws Exception {
|
||||
|
||||
Path familyDir = new Path(loadPath, Bytes.toString(family));
|
||||
fs.mkdirs(familyDir);
|
||||
int hfileIdx = 0;
|
||||
for (byte[][] range : hfileRanges) {
|
||||
byte[] from = range[0];
|
||||
byte[] to = range[1];
|
||||
createHFile(new Path(familyDir, "hfile_"+(hfileIdx++)),
|
||||
family, qualifier, from, to, numRowsPerRange);
|
||||
}
|
||||
//set global read so RegionServer can move it
|
||||
setPermission(loadPath, FsPermission.valueOf("-rwxrwxrwx"));
|
||||
|
||||
HTable table = new HTable(conf, tableName);
|
||||
TEST_UTIL.waitTableAvailable(tableName, 30000);
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
|
||||
loader.doBulkLoad(loadPath, table);
|
||||
}
|
||||
|
||||
public void setPermission(Path dir, FsPermission perm) throws IOException {
|
||||
if(!fs.getFileStatus(dir).isDir()) {
|
||||
fs.setPermission(dir,perm);
|
||||
}
|
||||
else {
|
||||
for(FileStatus el : fs.listStatus(dir)) {
|
||||
fs.setPermission(el.getPath(), perm);
|
||||
setPermission(el.getPath() , perm);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppend() throws Exception {
|
||||
|
||||
|
|
Loading…
Reference in New Issue