HBASE-12533: staging directories are not deleted after secure bulk load

This commit is contained in:
Jeffrey Zhong 2014-11-26 16:23:11 -08:00
parent b2cdeacc8c
commit f0d95e7f11
3 changed files with 48 additions and 48 deletions

View File

@ -26,6 +26,7 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -54,33 +55,29 @@ public class SecureBulkLoadClient {
public String prepareBulkLoad(final TableName tableName) throws IOException { public String prepareBulkLoad(final TableName tableName) throws IOException {
try { try {
return CoprocessorRpcChannel channel = table.coprocessorService(HConstants.EMPTY_START_ROW);
table.coprocessorService(SecureBulkLoadProtos.SecureBulkLoadService.class, SecureBulkLoadProtos.SecureBulkLoadService instance =
EMPTY_START_ROW, ProtobufUtil.newServiceStub(SecureBulkLoadProtos.SecureBulkLoadService.class, channel);
LAST_ROW,
new Batch.Call<SecureBulkLoadProtos.SecureBulkLoadService,String>() {
@Override
public String call(SecureBulkLoadProtos.SecureBulkLoadService instance) throws IOException {
ServerRpcController controller = new ServerRpcController();
BlockingRpcCallback<SecureBulkLoadProtos.PrepareBulkLoadResponse> rpcCallback = ServerRpcController controller = new ServerRpcController();
new BlockingRpcCallback<SecureBulkLoadProtos.PrepareBulkLoadResponse>();
SecureBulkLoadProtos.PrepareBulkLoadRequest request = BlockingRpcCallback<SecureBulkLoadProtos.PrepareBulkLoadResponse> rpcCallback =
SecureBulkLoadProtos.PrepareBulkLoadRequest.newBuilder() new BlockingRpcCallback<SecureBulkLoadProtos.PrepareBulkLoadResponse>();
.setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
instance.prepareBulkLoad(controller, SecureBulkLoadProtos.PrepareBulkLoadRequest request =
request, SecureBulkLoadProtos.PrepareBulkLoadRequest.newBuilder()
rpcCallback); .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
SecureBulkLoadProtos.PrepareBulkLoadResponse response = rpcCallback.get(); instance.prepareBulkLoad(controller,
if (controller.failedOnException()) { request,
throw controller.getFailedOn(); rpcCallback);
}
return response.getBulkToken(); SecureBulkLoadProtos.PrepareBulkLoadResponse response = rpcCallback.get();
} if (controller.failedOnException()) {
}).entrySet().iterator().next().getValue(); throw controller.getFailedOn();
}
return response.getBulkToken();
} catch (Throwable throwable) { } catch (Throwable throwable) {
throw new IOException(throwable); throw new IOException(throwable);
} }
@ -88,32 +85,26 @@ public class SecureBulkLoadClient {
public void cleanupBulkLoad(final String bulkToken) throws IOException { public void cleanupBulkLoad(final String bulkToken) throws IOException {
try { try {
table.coprocessorService(SecureBulkLoadProtos.SecureBulkLoadService.class, CoprocessorRpcChannel channel = table.coprocessorService(HConstants.EMPTY_START_ROW);
EMPTY_START_ROW, SecureBulkLoadProtos.SecureBulkLoadService instance =
LAST_ROW, ProtobufUtil.newServiceStub(SecureBulkLoadProtos.SecureBulkLoadService.class, channel);
new Batch.Call<SecureBulkLoadProtos.SecureBulkLoadService, String>() {
@Override ServerRpcController controller = new ServerRpcController();
public String call(SecureBulkLoadProtos.SecureBulkLoadService instance) throws IOException {
ServerRpcController controller = new ServerRpcController();
BlockingRpcCallback<SecureBulkLoadProtos.CleanupBulkLoadResponse> rpcCallback = BlockingRpcCallback<SecureBulkLoadProtos.CleanupBulkLoadResponse> rpcCallback =
new BlockingRpcCallback<SecureBulkLoadProtos.CleanupBulkLoadResponse>(); new BlockingRpcCallback<SecureBulkLoadProtos.CleanupBulkLoadResponse>();
SecureBulkLoadProtos.CleanupBulkLoadRequest request = SecureBulkLoadProtos.CleanupBulkLoadRequest request =
SecureBulkLoadProtos.CleanupBulkLoadRequest.newBuilder() SecureBulkLoadProtos.CleanupBulkLoadRequest.newBuilder()
.setBulkToken(bulkToken).build(); .setBulkToken(bulkToken).build();
instance.cleanupBulkLoad(controller, instance.cleanupBulkLoad(controller,
request, request,
rpcCallback); rpcCallback);
if (controller.failedOnException()) { if (controller.failedOnException()) {
throw controller.getFailedOn(); throw controller.getFailedOn();
} }
return null;
}
});
} catch (Throwable throwable) { } catch (Throwable throwable) {
throw new IOException(throwable); throw new IOException(throwable);
} }

View File

@ -199,10 +199,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
} }
} }
fs.delete(createStagingDir(baseStagingDir, fs.delete(new Path(request.getBulkToken()), true);
getActiveUser(),
new Path(request.getBulkToken()).getName()),
true);
done.run(CleanupBulkLoadResponse.newBuilder().build()); done.run(CleanupBulkLoadResponse.newBuilder().build());
} catch (IOException e) { } catch (IOException e) {
ResponseConverter.setControllerException(controller, e); ResponseConverter.setControllerException(controller, e);

View File

@ -27,6 +27,7 @@ import java.io.IOException;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -50,6 +51,7 @@ import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
/** /**
* Test cases for the "load" half of the HFileOutputFormat bulk load * Test cases for the "load" half of the HFileOutputFormat bulk load
@ -260,6 +262,16 @@ public class TestLoadIncrementalHFiles {
table.close(); table.close();
} }
// verify staging folder has been cleaned up
Path stagingBasePath = SecureBulkLoadUtil.getBaseStagingDir(util.getConfiguration());
if(fs.exists(stagingBasePath)) {
FileStatus[] files = fs.listStatus(stagingBasePath);
for(FileStatus file : files) {
assertTrue("Folder=" + file.getPath() + " is not cleaned up.",
file.getPath().getName() != "DONOTERASE");
}
}
util.deleteTable(tableName); util.deleteTable(tableName);
} }