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:
Zhihong Yu 2013-01-15 15:19:52 +00:00
parent 474b61483b
commit 7fb425daf4
14 changed files with 5069 additions and 34 deletions

View File

@ -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);
}

View File

@ -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);
}
}

View File

@ -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 {
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);
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();
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);
}
}
}
}
}
};

View File

@ -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;
}
}

View File

@ -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();

View File

@ -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,

View File

@ -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;
}
}
}

View File

@ -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));

View File

@ -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(

View File

@ -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);
}
}

View File

@ -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() {
}
}

View File

@ -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);

View File

@ -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,23 +207,30 @@ public class TestAccessController {
try {
user.runAs(action);
fail("Expected AccessDeniedException for user '" + user.getShortName() + "'");
} catch (RetriesExhaustedWithDetailsException e) {
// in case of batch operations, and put, the client assembles a
// RetriesExhaustedWithDetailsException instead of throwing an
// AccessDeniedException
} catch (IOException e) {
boolean isAccessDeniedException = false;
for (Throwable ex : e.getCauses()) {
if (ex instanceof ServiceException) {
ServiceException se = (ServiceException)ex;
if (se.getCause() != null && se.getCause() instanceof AccessDeniedException) {
if(e instanceof RetriesExhaustedWithDetailsException) {
// in case of batch operations, and put, the client assembles a
// RetriesExhaustedWithDetailsException instead of throwing an
// AccessDeniedException
for(Throwable ex : ((RetriesExhaustedWithDetailsException) e).getCauses()) {
if (ex instanceof AccessDeniedException) {
isAccessDeniedException = true;
break;
}
} else if (ex instanceof AccessDeniedException) {
isAccessDeniedException = true;
break;
}
}
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 {