HBASE-21779 Reimplement BulkLoadHFilesTool to use AsyncClusterConnection
This commit is contained in:
parent
72d9f8747f
commit
2182bfb942
|
@ -17,11 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.backup.mapreduce;
|
||||
|
||||
import static org.apache.hadoop.hbase.backup.util.BackupUtils.failed;
|
||||
import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -29,7 +27,7 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
|
||||
import org.apache.hadoop.hbase.backup.RestoreJob;
|
||||
import org.apache.hadoop.hbase.backup.util.BackupUtils;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -42,8 +40,7 @@ import org.slf4j.LoggerFactory;
|
|||
* HFiles which are aligned with a region boundaries of a table being
|
||||
* restored.
|
||||
*
|
||||
* The resulting HFiles then are loaded using HBase bulk load tool
|
||||
* {@link LoadIncrementalHFiles}
|
||||
* The resulting HFiles then are loaded using HBase bulk load tool {@link BulkLoadHFiles}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MapReduceRestoreJob implements RestoreJob {
|
||||
|
@ -88,23 +85,20 @@ public class MapReduceRestoreJob implements RestoreJob {
|
|||
};
|
||||
|
||||
int result;
|
||||
int loaderResult;
|
||||
try {
|
||||
|
||||
player.setConf(getConf());
|
||||
result = player.run(playerArgs);
|
||||
if (succeeded(result)) {
|
||||
// do bulk load
|
||||
LoadIncrementalHFiles loader = BackupUtils.createLoader(getConf());
|
||||
BulkLoadHFiles loader = BackupUtils.createLoader(getConf());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Restoring HFiles from directory " + bulkOutputPath);
|
||||
}
|
||||
String[] args = { bulkOutputPath.toString(), newTableNames[i].getNameAsString() };
|
||||
loaderResult = loader.run(args);
|
||||
|
||||
if (failed(loaderResult)) {
|
||||
throw new IOException("Can not restore from backup directory " + dirs
|
||||
+ " (check Hadoop and HBase logs). Bulk loader return code =" + loaderResult);
|
||||
if (loader.bulkLoad(newTableNames[i], bulkOutputPath).isEmpty()) {
|
||||
throw new IOException("Can not restore from backup directory " + dirs +
|
||||
" (check Hadoop and HBase logs). Bulk loader returns null");
|
||||
}
|
||||
} else {
|
||||
throw new IOException("Can not restore from backup directory " + dirs
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.util.List;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.client.Connection;
|
|||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.CommonFSUtils;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
|
@ -722,7 +721,7 @@ public final class BackupUtils {
|
|||
return result == 0;
|
||||
}
|
||||
|
||||
public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
|
||||
public static BulkLoadHFiles createLoader(Configuration config) {
|
||||
// set configuration for restore:
|
||||
// LoadIncrementalHFile needs more time
|
||||
// <name>hbase.rpc.timeout</name> <value>600000</value>
|
||||
|
@ -732,15 +731,9 @@ public final class BackupUtils {
|
|||
|
||||
// By default, it is 32 and loader will fail if # of files in any region exceed this
|
||||
// limit. Bad for snapshot restore.
|
||||
conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
|
||||
conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
|
||||
LoadIncrementalHFiles loader;
|
||||
try {
|
||||
loader = new LoadIncrementalHFiles(conf);
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
return loader;
|
||||
conf.setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
|
||||
conf.set(BulkLoadHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
|
||||
return BulkLoadHFiles.create(conf);
|
||||
}
|
||||
|
||||
public static String findMostRecentBackupId(String[] backupIds) {
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.util.Arrays;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -46,7 +45,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
|
@ -454,12 +453,12 @@ public class RestoreTool {
|
|||
}
|
||||
}
|
||||
}
|
||||
return LoadIncrementalHFiles.inferBoundaries(map);
|
||||
return BulkLoadHFilesTool.inferBoundaries(map);
|
||||
}
|
||||
|
||||
/**
|
||||
* Prepare the table for bulkload, most codes copied from
|
||||
* {@link LoadIncrementalHFiles#createTable(TableName, Path, Admin)}
|
||||
* Prepare the table for bulkload, most codes copied from {@code createTable} method in
|
||||
* {@code BulkLoadHFilesTool}.
|
||||
* @param conn connection
|
||||
* @param tableBackupPath path
|
||||
* @param tableName table name
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
|
|||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.tool.TestLoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.TestBulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.junit.Assert;
|
||||
|
@ -92,7 +92,7 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
|
|||
|
||||
int NB_ROWS2 = 20;
|
||||
LOG.debug("bulk loading into " + testName);
|
||||
int actual = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
|
||||
int actual = TestBulkLoadHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
|
||||
qualName, false, null, new byte[][][] {
|
||||
new byte[][]{ Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
|
||||
new byte[][]{ Bytes.toBytes("ddd"), Bytes.toBytes("ooo") },
|
||||
|
@ -105,7 +105,7 @@ public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
|
|||
assertTrue(checkSucceeded(backupIdIncMultiple));
|
||||
// #4 bulk load again
|
||||
LOG.debug("bulk loading into " + testName);
|
||||
int actual1 = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
|
||||
int actual1 = TestBulkLoadHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
|
||||
qualName, false, null,
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("ppp"), Bytes.toBytes("qqq") },
|
||||
new byte[][] { Bytes.toBytes("rrr"), Bytes.toBytes("sss") }, },
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -59,6 +60,7 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.Timer;
|
||||
|
||||
|
@ -695,4 +697,43 @@ public final class ConnectionUtils {
|
|||
metrics -> ResultStatsUtil.updateStats(metrics, serverName, regionName, regionLoadStats));
|
||||
});
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
interface Converter<D, I, S> {
|
||||
D convert(I info, S src) throws IOException;
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
interface RpcCall<RESP, REQ> {
|
||||
void call(ClientService.Interface stub, HBaseRpcController controller, REQ req,
|
||||
RpcCallback<RESP> done);
|
||||
}
|
||||
|
||||
static <REQ, PREQ, PRESP, RESP> CompletableFuture<RESP> call(HBaseRpcController controller,
|
||||
HRegionLocation loc, ClientService.Interface stub, REQ req,
|
||||
Converter<PREQ, byte[], REQ> reqConvert, RpcCall<PRESP, PREQ> rpcCall,
|
||||
Converter<RESP, HBaseRpcController, PRESP> respConverter) {
|
||||
CompletableFuture<RESP> future = new CompletableFuture<>();
|
||||
try {
|
||||
rpcCall.call(stub, controller, reqConvert.convert(loc.getRegion().getRegionName(), req),
|
||||
new RpcCallback<PRESP>() {
|
||||
|
||||
@Override
|
||||
public void run(PRESP resp) {
|
||||
if (controller.failed()) {
|
||||
future.completeExceptionally(controller.getFailed());
|
||||
} else {
|
||||
try {
|
||||
future.complete(respConverter.convert(controller, resp));
|
||||
} catch (IOException e) {
|
||||
future.completeExceptionally(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (IOException e) {
|
||||
future.completeExceptionally(e);
|
||||
}
|
||||
return future;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
|
||||
import org.apache.hadoop.hbase.client.ConnectionUtils.Converter;
|
||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||
|
@ -156,51 +157,12 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
|
|||
return conn.getRegionLocator(tableName);
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
private interface Converter<D, I, S> {
|
||||
D convert(I info, S src) throws IOException;
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
private interface RpcCall<RESP, REQ> {
|
||||
void call(ClientService.Interface stub, HBaseRpcController controller, REQ req,
|
||||
RpcCallback<RESP> done);
|
||||
}
|
||||
|
||||
private static <REQ, PREQ, PRESP, RESP> CompletableFuture<RESP> call(
|
||||
HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,
|
||||
Converter<PREQ, byte[], REQ> reqConvert, RpcCall<PRESP, PREQ> rpcCall,
|
||||
Converter<RESP, HBaseRpcController, PRESP> respConverter) {
|
||||
CompletableFuture<RESP> future = new CompletableFuture<>();
|
||||
try {
|
||||
rpcCall.call(stub, controller, reqConvert.convert(loc.getRegion().getRegionName(), req),
|
||||
new RpcCallback<PRESP>() {
|
||||
|
||||
@Override
|
||||
public void run(PRESP resp) {
|
||||
if (controller.failed()) {
|
||||
future.completeExceptionally(controller.getFailed());
|
||||
} else {
|
||||
try {
|
||||
future.complete(respConverter.convert(controller, resp));
|
||||
} catch (IOException e) {
|
||||
future.completeExceptionally(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (IOException e) {
|
||||
future.completeExceptionally(e);
|
||||
}
|
||||
return future;
|
||||
}
|
||||
|
||||
private static <REQ, RESP> CompletableFuture<RESP> mutate(HBaseRpcController controller,
|
||||
HRegionLocation loc, ClientService.Interface stub, REQ req,
|
||||
Converter<MutateRequest, byte[], REQ> reqConvert,
|
||||
Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {
|
||||
return call(controller, loc, stub, req, reqConvert, (s, c, r, done) -> s.mutate(c, r, done),
|
||||
respConverter);
|
||||
return ConnectionUtils.call(controller, loc, stub, req, reqConvert,
|
||||
(s, c, r, done) -> s.mutate(c, r, done), respConverter);
|
||||
}
|
||||
|
||||
private static <REQ> CompletableFuture<Void> voidMutate(HBaseRpcController controller,
|
||||
|
@ -247,7 +209,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
|
|||
|
||||
private CompletableFuture<Result> get(Get get, int replicaId) {
|
||||
return this.<Result, Get> newCaller(get, readRpcTimeoutNs)
|
||||
.action((controller, loc, stub) -> RawAsyncTableImpl
|
||||
.action((controller, loc, stub) -> ConnectionUtils
|
||||
.<Get, GetRequest, GetResponse, Result> call(controller, loc, stub, get,
|
||||
RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done),
|
||||
(c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))
|
||||
|
|
|
@ -1,150 +0,0 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import static org.apache.hadoop.hbase.HConstants.PRIORITY_UNSET;
|
||||
|
||||
/**
|
||||
* Client proxy for SecureBulkLoadProtocol
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SecureBulkLoadClient {
|
||||
private Table table;
|
||||
private final RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
public SecureBulkLoadClient(final Configuration conf, Table table) {
|
||||
this.table = table;
|
||||
this.rpcControllerFactory = new RpcControllerFactory(conf);
|
||||
}
|
||||
|
||||
public String prepareBulkLoad(final Connection conn) throws IOException {
|
||||
try {
|
||||
ClientServiceCallable<String> callable = new ClientServiceCallable<String>(conn,
|
||||
table.getName(), HConstants.EMPTY_START_ROW,
|
||||
this.rpcControllerFactory.newController(), PRIORITY_UNSET) {
|
||||
@Override
|
||||
protected String rpcCall() throws Exception {
|
||||
byte[] regionName = getLocation().getRegion().getRegionName();
|
||||
RegionSpecifier region =
|
||||
RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
|
||||
PrepareBulkLoadRequest request = PrepareBulkLoadRequest.newBuilder()
|
||||
.setTableName(ProtobufUtil.toProtoTableName(table.getName()))
|
||||
.setRegion(region).build();
|
||||
PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
|
||||
return response.getBulkToken();
|
||||
}
|
||||
};
|
||||
return RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
|
||||
.<String> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
|
||||
} catch (Throwable throwable) {
|
||||
throw new IOException(throwable);
|
||||
}
|
||||
}
|
||||
|
||||
public void cleanupBulkLoad(final Connection conn, final String bulkToken) throws IOException {
|
||||
try {
|
||||
ClientServiceCallable<Void> callable = new ClientServiceCallable<Void>(conn,
|
||||
table.getName(), HConstants.EMPTY_START_ROW, this.rpcControllerFactory.newController(), PRIORITY_UNSET) {
|
||||
@Override
|
||||
protected Void rpcCall() throws Exception {
|
||||
byte[] regionName = getLocation().getRegion().getRegionName();
|
||||
RegionSpecifier region = RequestConverter.buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
CleanupBulkLoadRequest request =
|
||||
CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bulkToken).build();
|
||||
getStub().cleanupBulkLoad(null, request);
|
||||
return null;
|
||||
}
|
||||
};
|
||||
RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
|
||||
.<Void> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
|
||||
} catch (Throwable throwable) {
|
||||
throw new IOException(throwable);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Securely bulk load a list of HFiles using client protocol.
|
||||
*
|
||||
* @param client
|
||||
* @param familyPaths
|
||||
* @param regionName
|
||||
* @param assignSeqNum
|
||||
* @param userToken
|
||||
* @param bulkToken
|
||||
* @return true if all are loaded
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client,
|
||||
final List<Pair<byte[], String>> familyPaths,
|
||||
final byte[] regionName, boolean assignSeqNum,
|
||||
final Token<?> userToken, final String bulkToken) throws IOException {
|
||||
return secureBulkLoadHFiles(client, familyPaths, regionName, assignSeqNum, userToken, bulkToken,
|
||||
false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Securely bulk load a list of HFiles using client protocol.
|
||||
*
|
||||
* @param client
|
||||
* @param familyPaths
|
||||
* @param regionName
|
||||
* @param assignSeqNum
|
||||
* @param userToken
|
||||
* @param bulkToken
|
||||
* @param copyFiles
|
||||
* @return true if all are loaded
|
||||
* @throws IOException
|
||||
*/
|
||||
public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client,
|
||||
final List<Pair<byte[], String>> familyPaths,
|
||||
final byte[] regionName, boolean assignSeqNum,
|
||||
final Token<?> userToken, final String bulkToken, boolean copyFiles) throws IOException {
|
||||
BulkLoadHFileRequest request =
|
||||
RequestConverter.buildBulkLoadHFileRequest(familyPaths, regionName, assignSeqNum,
|
||||
userToken, bulkToken, copyFiles);
|
||||
|
||||
try {
|
||||
BulkLoadHFileResponse response = client.bulkLoadHFile(null, request);
|
||||
return response.getLoaded();
|
||||
} catch (Exception se) {
|
||||
throw ProtobufUtil.handleRemoteException(se);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.client.Consistency;
|
|||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
|
@ -60,7 +59,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver;
|
|||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.RegionSplitter;
|
||||
|
@ -86,6 +85,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
|
||||
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
|
||||
|
@ -292,24 +292,18 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
|
|||
|
||||
// Set where to place the hfiles.
|
||||
FileOutputFormat.setOutputPath(job, p);
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Admin admin = conn.getAdmin();
|
||||
Table table = conn.getTable(getTablename());
|
||||
RegionLocator regionLocator = conn.getRegionLocator(getTablename())) {
|
||||
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin();
|
||||
RegionLocator regionLocator = conn.getRegionLocator(getTablename())) {
|
||||
// Configure the partitioner and other things needed for HFileOutputFormat.
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, admin.getDescriptor(getTablename()),
|
||||
regionLocator);
|
||||
// Run the job making sure it works.
|
||||
assertEquals(true, job.waitForCompletion(true));
|
||||
|
||||
// Create a new loader.
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
|
||||
|
||||
// Load the HFiles in.
|
||||
loader.doBulkLoad(p, admin, table, regionLocator);
|
||||
}
|
||||
|
||||
// Create a new loader.
|
||||
BulkLoadHFiles loader = BulkLoadHFiles.create(conf);
|
||||
// Load the HFiles in.
|
||||
loader.bulkLoad(getTablename(), p);
|
||||
// Delete the files.
|
||||
util.getTestFileSystem().delete(p, true);
|
||||
}
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.util.Iterator;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
|
@ -60,7 +59,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Validate ImportTsv + LoadIncrementalHFiles on a distributed cluster.
|
||||
* Validate ImportTsv + BulkLoadFiles on a distributed cluster.
|
||||
*/
|
||||
@Category(IntegrationTests.class)
|
||||
public class IntegrationTestImportTsv extends Configured implements Tool {
|
||||
|
@ -141,8 +140,8 @@ public class IntegrationTestImportTsv extends Configured implements Tool {
|
|||
|
||||
String[] args = { hfiles.toString(), tableName.getNameAsString() };
|
||||
LOG.info(format("Running LoadIncrememntalHFiles with args: %s", Arrays.asList(args)));
|
||||
assertEquals("Loading HFiles failed.",
|
||||
0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args));
|
||||
assertEquals("Loading HFiles failed.", 0,
|
||||
ToolRunner.run(new BulkLoadHFilesTool(getConf()), args));
|
||||
|
||||
Table table = null;
|
||||
Scan scan = new Scan() {{
|
||||
|
|
|
@ -22,28 +22,28 @@ import java.io.IOException;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.Importer;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import.Importer;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Tool used to copy a table to another one which can be on a different setup.
|
||||
|
@ -416,13 +416,12 @@ public class CopyTable extends Configured implements Tool {
|
|||
int code = 0;
|
||||
if (bulkload) {
|
||||
LOG.info("Trying to bulk load data to destination table: " + dstTableName);
|
||||
LOG.info("command: ./bin/hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles {} {}",
|
||||
LOG.info("command: ./bin/hbase {} {} {}", BulkLoadHFilesTool.NAME,
|
||||
this.bulkloadDir.toString(), this.dstTableName);
|
||||
code = new LoadIncrementalHFiles(this.getConf())
|
||||
.run(new String[] { this.bulkloadDir.toString(), this.dstTableName });
|
||||
if (code == 0) {
|
||||
// bulkloadDir is deleted only LoadIncrementalHFiles was successful so that one can rerun
|
||||
// LoadIncrementalHFiles.
|
||||
if (!BulkLoadHFiles.create(getConf()).bulkLoad(TableName.valueOf(dstTableName), bulkloadDir)
|
||||
.isEmpty()) {
|
||||
// bulkloadDir is deleted only BulkLoadHFiles was successful so that one can rerun
|
||||
// BulkLoadHFiles.
|
||||
FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
|
||||
if (!fs.delete(this.bulkloadDir, true)) {
|
||||
LOG.error("Deleting folder " + bulkloadDir + " failed!");
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.hadoop.mapreduce.Partitioner;
|
|||
*
|
||||
* <p>This class is not suitable as partitioner creating hfiles
|
||||
* for incremental bulk loads as region spread will likely change between time of
|
||||
* hfile creation and load time. See {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles}
|
||||
* hfile creation and load time. See {@link org.apache.hadoop.hbase.tool.BulkLoadHFiles}
|
||||
* and <a href="http://hbase.apache.org/book.html#arch.bulk.load">Bulk Load</a>.</p>
|
||||
*
|
||||
* @param <KEY> The type of the key.
|
||||
|
|
|
@ -91,7 +91,7 @@ import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem;
|
|||
import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
@ -708,18 +708,17 @@ public class TestHFileOutputFormat2 {
|
|||
}
|
||||
Table currentTable = allTables.get(tableNameStr);
|
||||
TableName currentTableName = currentTable.getName();
|
||||
new LoadIncrementalHFiles(conf).doBulkLoad(tableDir, admin, currentTable, singleTableInfo
|
||||
.getRegionLocator());
|
||||
BulkLoadHFiles.create(conf).bulkLoad(currentTableName, tableDir);
|
||||
|
||||
// Ensure data shows up
|
||||
int expectedRows = 0;
|
||||
if (putSortReducer) {
|
||||
// no rows should be extracted
|
||||
assertEquals("LoadIncrementalHFiles should put expected data in table", expectedRows,
|
||||
assertEquals("BulkLoadHFiles should put expected data in table", expectedRows,
|
||||
util.countRows(currentTable));
|
||||
} else {
|
||||
expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
|
||||
assertEquals("LoadIncrementalHFiles should put expected data in table", expectedRows,
|
||||
assertEquals("BulkLoadHFiles should put expected data in table", expectedRows,
|
||||
util.countRows(currentTable));
|
||||
Scan scan = new Scan();
|
||||
ResultScanner results = currentTable.getScanner(scan);
|
||||
|
@ -1251,12 +1250,12 @@ public class TestHFileOutputFormat2 {
|
|||
runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(table
|
||||
.getDescriptor(), conn.getRegionLocator(TABLE_NAMES[0]))), testDir, false);
|
||||
// Perform the actual load
|
||||
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, locator);
|
||||
BulkLoadHFiles.create(conf).bulkLoad(table.getName(), testDir);
|
||||
}
|
||||
|
||||
// Ensure data shows up
|
||||
int expectedRows = 2 * NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
|
||||
assertEquals("LoadIncrementalHFiles should put expected data in table",
|
||||
assertEquals("BulkLoadHFiles should put expected data in table",
|
||||
expectedRows, util.countRows(table));
|
||||
|
||||
// should have a second StoreFile now
|
||||
|
@ -1341,15 +1340,16 @@ public class TestHFileOutputFormat2 {
|
|||
true);
|
||||
|
||||
RegionLocator regionLocator = conn.getRegionLocator(TABLE_NAMES[0]);
|
||||
runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(table
|
||||
.getDescriptor(), regionLocator)), testDir, false);
|
||||
runIncrementalPELoad(conf,
|
||||
Arrays.asList(new HFileOutputFormat2.TableInfo(table.getDescriptor(), regionLocator)),
|
||||
testDir, false);
|
||||
|
||||
// Perform the actual load
|
||||
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, regionLocator);
|
||||
BulkLoadHFiles.create(conf).bulkLoad(table.getName(), testDir);
|
||||
|
||||
// Ensure data shows up
|
||||
int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
|
||||
assertEquals("LoadIncrementalHFiles should put expected data in table",
|
||||
assertEquals("BulkLoadHFiles should put expected data in table",
|
||||
expectedRows + 1, util.countRows(table));
|
||||
|
||||
// should have a second StoreFile now
|
||||
|
|
|
@ -23,7 +23,9 @@ import org.apache.hadoop.hbase.RegionLocations;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
|
||||
|
@ -66,4 +68,21 @@ public interface AsyncClusterConnection extends AsyncConnection {
|
|||
*/
|
||||
CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
|
||||
boolean reload);
|
||||
|
||||
/**
|
||||
* Return the token for this bulk load.
|
||||
*/
|
||||
CompletableFuture<String> prepareBulkLoad(TableName tableName);
|
||||
|
||||
/**
|
||||
* Securely bulk load a list of HFiles.
|
||||
* @param row used to locate the region
|
||||
*/
|
||||
CompletableFuture<Boolean> bulkLoad(TableName tableName, List<Pair<byte[], String>> familyPaths,
|
||||
byte[] row, boolean assignSeqNum, Token<?> userToken, String bulkToken, boolean copyFiles);
|
||||
|
||||
/**
|
||||
* Clean up after finishing bulk load, no matter success or not.
|
||||
*/
|
||||
CompletableFuture<Void> cleanupBulkLoad(TableName tableName, String bulkToken);
|
||||
}
|
||||
|
|
|
@ -21,15 +21,28 @@ import java.net.SocketAddress;
|
|||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
||||
|
||||
/**
|
||||
* The implementation of AsyncClusterConnection.
|
||||
|
@ -77,4 +90,46 @@ class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements AsyncClu
|
|||
boolean reload) {
|
||||
return getLocator().getRegionLocations(tableName, row, RegionLocateType.CURRENT, reload, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<String> prepareBulkLoad(TableName tableName) {
|
||||
return callerFactory.<String> single().table(tableName).row(HConstants.EMPTY_START_ROW)
|
||||
.action((controller, loc, stub) -> ConnectionUtils
|
||||
.<TableName, PrepareBulkLoadRequest, PrepareBulkLoadResponse, String> call(controller, loc,
|
||||
stub, tableName, (rn, tn) -> {
|
||||
RegionSpecifier region =
|
||||
RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, rn);
|
||||
return PrepareBulkLoadRequest.newBuilder()
|
||||
.setTableName(ProtobufUtil.toProtoTableName(tn)).setRegion(region).build();
|
||||
}, (s, c, req, done) -> s.prepareBulkLoad(c, req, done),
|
||||
(c, resp) -> resp.getBulkToken()))
|
||||
.call();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Boolean> bulkLoad(TableName tableName,
|
||||
List<Pair<byte[], String>> familyPaths, byte[] row, boolean assignSeqNum, Token<?> userToken,
|
||||
String bulkToken, boolean copyFiles) {
|
||||
return callerFactory.<Boolean> single().table(tableName).row(row)
|
||||
.action((controller, loc, stub) -> ConnectionUtils
|
||||
.<Void, BulkLoadHFileRequest, BulkLoadHFileResponse, Boolean> call(controller, loc, stub,
|
||||
null,
|
||||
(rn, nil) -> RequestConverter.buildBulkLoadHFileRequest(familyPaths, rn, assignSeqNum,
|
||||
userToken, bulkToken, copyFiles),
|
||||
(s, c, req, done) -> s.bulkLoadHFile(c, req, done), (c, resp) -> resp.getLoaded()))
|
||||
.call();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> cleanupBulkLoad(TableName tableName, String bulkToken) {
|
||||
return callerFactory.<Void> single().table(tableName).row(HConstants.EMPTY_START_ROW)
|
||||
.action((controller, loc, stub) -> ConnectionUtils
|
||||
.<String, CleanupBulkLoadRequest, CleanupBulkLoadResponse, Void> call(controller, loc, stub,
|
||||
bulkToken, (rn, bt) -> {
|
||||
RegionSpecifier region =
|
||||
RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, rn);
|
||||
return CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bt).build();
|
||||
}, (s, c, req, done) -> s.cleanupBulkLoad(c, req, done), (c, resp) -> null))
|
||||
.call();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,9 +19,11 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.net.SocketAddress;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.FutureUtils;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
|
@ -30,6 +32,9 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@InterfaceAudience.Private
|
||||
public final class ClusterConnectionFactory {
|
||||
|
||||
public static final String HBASE_SERVER_CLUSTER_CONNECTION_IMPL =
|
||||
"hbase.server.cluster.connection.impl";
|
||||
|
||||
private ClusterConnectionFactory() {
|
||||
}
|
||||
|
||||
|
@ -46,6 +51,15 @@ public final class ClusterConnectionFactory {
|
|||
SocketAddress localAddress, User user) throws IOException {
|
||||
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
|
||||
String clusterId = FutureUtils.get(registry.getClusterId());
|
||||
return new AsyncClusterConnectionImpl(conf, registry, clusterId, localAddress, user);
|
||||
Class<? extends AsyncClusterConnection> clazz =
|
||||
conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class,
|
||||
AsyncClusterConnection.class);
|
||||
try {
|
||||
return user
|
||||
.runAs((PrivilegedExceptionAction<? extends AsyncClusterConnection>) () -> ReflectionUtils
|
||||
.newInstance(clazz, conf, registry, clusterId, localAddress, user));
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,77 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* 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 java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Tool to load the output of HFileOutputFormat into an existing table.
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
|
||||
* {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles} instead.
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
|
||||
justification = "Temporary glue. To be removed")
|
||||
@Deprecated
|
||||
@InterfaceAudience.Public
|
||||
public class LoadIncrementalHFiles extends org.apache.hadoop.hbase.tool.LoadIncrementalHFiles {
|
||||
|
||||
/**
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
|
||||
* {@link org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem} instead.
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
|
||||
justification = "Temporary glue. To be removed")
|
||||
@Deprecated
|
||||
@InterfaceAudience.Public
|
||||
public static class LoadQueueItem
|
||||
extends org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem {
|
||||
|
||||
public LoadQueueItem(byte[] family, Path hfilePath) {
|
||||
super(family, hfilePath);
|
||||
}
|
||||
}
|
||||
|
||||
public LoadIncrementalHFiles(Configuration conf) {
|
||||
super(conf);
|
||||
}
|
||||
|
||||
public Map<LoadQueueItem, ByteBuffer> run(String dirPath, Map<byte[], List<Path>> map,
|
||||
TableName tableName) throws IOException {
|
||||
Map<org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem, ByteBuffer> originRet;
|
||||
if (dirPath != null) {
|
||||
originRet = run(dirPath, tableName);
|
||||
} else {
|
||||
originRet = run(map, tableName);
|
||||
}
|
||||
Map<LoadQueueItem, ByteBuffer> ret = new HashMap<>();
|
||||
originRet.forEach((k, v) -> {
|
||||
ret.put(new LoadQueueItem(k.getFamily(), k.getFilePath()), v);
|
||||
});
|
||||
return ret;
|
||||
}
|
||||
}
|
|
@ -40,7 +40,6 @@ import java.util.TreeMap;
|
|||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -82,8 +81,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreScanner;
|
||||
import org.apache.hadoop.hbase.security.EncryptionUtil;
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -91,6 +89,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* An implementation of {@link MobCompactor} that compacts the mob files in partitions.
|
||||
*/
|
||||
|
@ -675,7 +675,7 @@ public class PartitionedMobCompactor extends MobCompactor {
|
|||
cleanupTmpMobFile = false;
|
||||
cleanupCommittedMobFile = true;
|
||||
// bulkload the ref file
|
||||
bulkloadRefFile(connection, table, bulkloadPathOfPartition, filePath.getName());
|
||||
bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName());
|
||||
cleanupCommittedMobFile = false;
|
||||
newFiles.add(new Path(mobFamilyDir, filePath.getName()));
|
||||
}
|
||||
|
@ -818,21 +818,16 @@ public class PartitionedMobCompactor extends MobCompactor {
|
|||
|
||||
/**
|
||||
* Bulkloads the current file.
|
||||
*
|
||||
* @param connection to use to get admin/RegionLocator
|
||||
* @param table The current table.
|
||||
* @param tableName The table to load into.
|
||||
* @param bulkloadDirectory The path of bulkload directory.
|
||||
* @param fileName The current file name.
|
||||
* @throws IOException if IO failure is encountered
|
||||
*/
|
||||
private void bulkloadRefFile(Connection connection, Table table, Path bulkloadDirectory,
|
||||
String fileName)
|
||||
private void bulkloadRefFile(TableName tableName, Path bulkloadDirectory, String fileName)
|
||||
throws IOException {
|
||||
// bulkload the ref file
|
||||
try {
|
||||
LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
|
||||
bulkload.doBulkLoad(bulkloadDirectory, connection.getAdmin(), table,
|
||||
connection.getRegionLocator(table.getName()));
|
||||
BulkLoadHFiles.create(conf).bulkLoad(tableName, bulkloadDirectory);
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
|
|
@ -1,17 +1,22 @@
|
|||
/*
|
||||
* 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.
|
||||
/**
|
||||
* 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.replication.regionserver;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
|
@ -30,33 +35,32 @@ import java.util.concurrent.Future;
|
|||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem;
|
||||
import org.apache.hadoop.hbase.client.AsyncClusterConnection;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles.LoadQueueItem;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local
|
||||
* staging directory and then it will use ({@link LoadIncrementalHFiles} to prepare a collection of
|
||||
* staging directory and then it will use ({@link BulkLoadHFiles} to prepare a collection of
|
||||
* {@link LoadQueueItem} which will finally be loaded(replicated) into the table of this cluster.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
|
@ -82,7 +86,7 @@ public class HFileReplicator {
|
|||
private FsDelegationToken fsDelegationToken;
|
||||
private UserProvider userProvider;
|
||||
private Configuration conf;
|
||||
private Connection connection;
|
||||
private AsyncClusterConnection connection;
|
||||
private Path hbaseStagingDir;
|
||||
private ThreadPoolExecutor exec;
|
||||
private int maxCopyThreads;
|
||||
|
@ -91,7 +95,7 @@ public class HFileReplicator {
|
|||
public HFileReplicator(Configuration sourceClusterConf,
|
||||
String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath,
|
||||
Map<String, List<Pair<byte[], List<String>>>> tableQueueMap, Configuration conf,
|
||||
Connection connection) throws IOException {
|
||||
AsyncClusterConnection connection) throws IOException {
|
||||
this.sourceClusterConf = sourceClusterConf;
|
||||
this.sourceBaseNamespaceDirPath = sourceBaseNamespaceDirPath;
|
||||
this.sourceHFileArchiveDirPath = sourceHFileArchiveDirPath;
|
||||
|
@ -128,96 +132,61 @@ public class HFileReplicator {
|
|||
String tableNameString = tableStagingDir.getKey();
|
||||
Path stagingDir = tableStagingDir.getValue();
|
||||
|
||||
LoadIncrementalHFiles loadHFiles = null;
|
||||
try {
|
||||
loadHFiles = new LoadIncrementalHFiles(conf);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to initialize LoadIncrementalHFiles for replicating bulk loaded"
|
||||
+ " data.", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
Configuration newConf = HBaseConfiguration.create(conf);
|
||||
newConf.set(LoadIncrementalHFiles.CREATE_TABLE_CONF_KEY, "no");
|
||||
loadHFiles.setConf(newConf);
|
||||
|
||||
TableName tableName = TableName.valueOf(tableNameString);
|
||||
Table table = this.connection.getTable(tableName);
|
||||
|
||||
// Prepare collection of queue of hfiles to be loaded(replicated)
|
||||
Deque<LoadQueueItem> queue = new LinkedList<>();
|
||||
loadHFiles.prepareHFileQueue(stagingDir, table, queue, false);
|
||||
BulkLoadHFilesTool.prepareHFileQueue(conf, connection, tableName, stagingDir, queue, false,
|
||||
false);
|
||||
|
||||
if (queue.isEmpty()) {
|
||||
LOG.warn("Replication process did not find any files to replicate in directory "
|
||||
+ stagingDir.toUri());
|
||||
return null;
|
||||
}
|
||||
|
||||
try (RegionLocator locator = connection.getRegionLocator(tableName)) {
|
||||
|
||||
fsDelegationToken.acquireDelegationToken(sinkFs);
|
||||
|
||||
// Set the staging directory which will be used by LoadIncrementalHFiles for loading the
|
||||
// data
|
||||
loadHFiles.setBulkToken(stagingDir.toString());
|
||||
|
||||
doBulkLoad(loadHFiles, table, queue, locator, maxRetries);
|
||||
fsDelegationToken.acquireDelegationToken(sinkFs);
|
||||
try {
|
||||
doBulkLoad(conf, tableName, stagingDir, queue, maxRetries);
|
||||
} finally {
|
||||
cleanup(stagingDir.toString(), table);
|
||||
cleanup(stagingDir);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private void doBulkLoad(LoadIncrementalHFiles loadHFiles, Table table,
|
||||
Deque<LoadQueueItem> queue, RegionLocator locator, int maxRetries) throws IOException {
|
||||
int count = 0;
|
||||
Pair<byte[][], byte[][]> startEndKeys;
|
||||
while (!queue.isEmpty()) {
|
||||
// need to reload split keys each iteration.
|
||||
startEndKeys = locator.getStartEndKeys();
|
||||
private void doBulkLoad(Configuration conf, TableName tableName, Path stagingDir,
|
||||
Deque<LoadQueueItem> queue, int maxRetries) throws IOException {
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf);
|
||||
// Set the staging directory which will be used by BulkLoadHFilesTool for loading the data
|
||||
loader.setBulkToken(stagingDir.toString());
|
||||
for (int count = 0; !queue.isEmpty(); count++) {
|
||||
if (count != 0) {
|
||||
LOG.warn("Error occurred while replicating HFiles, retry attempt " + count + " with "
|
||||
+ queue.size() + " files still remaining to replicate.");
|
||||
LOG.warn("Error occurred while replicating HFiles, retry attempt " + count + " with " +
|
||||
queue.size() + " files still remaining to replicate.");
|
||||
}
|
||||
|
||||
if (maxRetries != 0 && count >= maxRetries) {
|
||||
throw new IOException("Retry attempted " + count
|
||||
+ " times without completing, bailing out.");
|
||||
throw new IOException(
|
||||
"Retry attempted " + count + " times without completing, bailing out.");
|
||||
}
|
||||
count++;
|
||||
|
||||
// Try bulk load
|
||||
loadHFiles.loadHFileQueue(table, connection, queue, startEndKeys);
|
||||
loader.loadHFileQueue(connection, tableName, queue, false);
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanup(String stagingDir, Table table) {
|
||||
private void cleanup(Path stagingDir) {
|
||||
// Release the file system delegation token
|
||||
fsDelegationToken.releaseDelegationToken();
|
||||
// Delete the staging directory
|
||||
if (stagingDir != null) {
|
||||
try {
|
||||
sinkFs.delete(new Path(stagingDir), true);
|
||||
sinkFs.delete(stagingDir, true);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to delete the staging directory " + stagingDir, e);
|
||||
}
|
||||
}
|
||||
// Do not close the file system
|
||||
|
||||
/*
|
||||
* if (sinkFs != null) { try { sinkFs.close(); } catch (IOException e) { LOG.warn(
|
||||
* "Failed to close the file system"); } }
|
||||
*/
|
||||
|
||||
// Close the table
|
||||
if (table != null) {
|
||||
try {
|
||||
table.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to close the table.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, Path> copyHFilesToStagingDir() throws IOException {
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
|
@ -27,8 +26,9 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.TreeMap;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -40,16 +40,18 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.AsyncClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.AsyncTable;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.client.Row;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationUtils;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FutureUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -83,7 +85,7 @@ public class ReplicationSink {
|
|||
private final Configuration conf;
|
||||
// Volatile because of note in here -- look for double-checked locking:
|
||||
// http://www.oracle.com/technetwork/articles/javase/bloch-effective-08-qa-140880.html
|
||||
private volatile Connection sharedConn;
|
||||
private volatile AsyncClusterConnection sharedConn;
|
||||
private final MetricsSink metrics;
|
||||
private final AtomicLong totalReplicatedEdits = new AtomicLong();
|
||||
private final Object sharedConnLock = new Object();
|
||||
|
@ -390,37 +392,34 @@ public class ReplicationSink {
|
|||
* Do the changes and handle the pool
|
||||
* @param tableName table to insert into
|
||||
* @param allRows list of actions
|
||||
* @throws IOException
|
||||
*/
|
||||
private void batch(TableName tableName, Collection<List<Row>> allRows) throws IOException {
|
||||
if (allRows.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
Connection connection = getConnection();
|
||||
try (Table table = connection.getTable(tableName)) {
|
||||
for (List<Row> rows : allRows) {
|
||||
table.batch(rows, null);
|
||||
}
|
||||
} catch (RetriesExhaustedWithDetailsException rewde) {
|
||||
for (Throwable ex : rewde.getCauses()) {
|
||||
if (ex instanceof TableNotFoundException) {
|
||||
AsyncTable<?> table = getConnection().getTable(tableName);
|
||||
List<Future<?>> futures = allRows.stream().map(table::batchAll).collect(Collectors.toList());
|
||||
for (Future<?> future : futures) {
|
||||
try {
|
||||
FutureUtils.get(future);
|
||||
} catch (RetriesExhaustedException e) {
|
||||
if (e.getCause() instanceof TableNotFoundException) {
|
||||
throw new TableNotFoundException("'" + tableName + "'");
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
throw rewde;
|
||||
} catch (InterruptedException ix) {
|
||||
throw (InterruptedIOException) new InterruptedIOException().initCause(ix);
|
||||
}
|
||||
}
|
||||
|
||||
private Connection getConnection() throws IOException {
|
||||
private AsyncClusterConnection getConnection() throws IOException {
|
||||
// See https://en.wikipedia.org/wiki/Double-checked_locking
|
||||
Connection connection = sharedConn;
|
||||
AsyncClusterConnection connection = sharedConn;
|
||||
if (connection == null) {
|
||||
synchronized (sharedConnLock) {
|
||||
connection = sharedConn;
|
||||
if (connection == null) {
|
||||
connection = ConnectionFactory.createConnection(conf);
|
||||
connection = ClusterConnectionFactory.createAsyncClusterConnection(conf, null,
|
||||
UserProvider.instantiate(conf).getCurrent());
|
||||
sharedConn = connection;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/*
|
||||
/**
|
||||
* 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
|
||||
|
@ -18,9 +18,10 @@
|
|||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.AsyncConnection;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Implementations are installed on a Replication Sink called from inside
|
||||
|
@ -36,6 +37,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
* source-side.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
|
||||
@InterfaceStability.Evolving
|
||||
public interface WALEntrySinkFilter {
|
||||
/**
|
||||
* Name of configuration to set with name of implementing WALEntrySinkFilter class.
|
||||
|
@ -46,7 +48,7 @@ public interface WALEntrySinkFilter {
|
|||
* Called after Construction.
|
||||
* Use passed Connection to keep any context the filter might need.
|
||||
*/
|
||||
void init(Connection connection);
|
||||
void init(AsyncConnection conn);
|
||||
|
||||
/**
|
||||
* @param table Table edit is destined for.
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -126,6 +126,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationUtils;
|
||||
import org.apache.hadoop.hbase.security.AccessDeniedException;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
||||
import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
|
||||
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
|
||||
import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
|
||||
|
@ -3562,14 +3563,12 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
}
|
||||
|
||||
public void dumpSidelinedRegions(Map<Path, HbckInfo> regions) {
|
||||
for (Map.Entry<Path, HbckInfo> entry: regions.entrySet()) {
|
||||
for (Map.Entry<Path, HbckInfo> entry : regions.entrySet()) {
|
||||
TableName tableName = entry.getValue().getTableName();
|
||||
Path path = entry.getKey();
|
||||
errors.print("This sidelined region dir should be bulk loaded: "
|
||||
+ path.toString());
|
||||
errors.print("Bulk load command looks like: "
|
||||
+ "hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles "
|
||||
+ path.toUri().getPath() + " "+ tableName);
|
||||
errors.print("This sidelined region dir should be bulk loaded: " + path.toString());
|
||||
errors.print("Bulk load command looks like: " + BulkLoadHFilesTool.NAME + " " +
|
||||
path.toUri().getPath() + " " + tableName);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,155 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
|
||||
|
||||
/**
|
||||
* Can be overridden in UT if you only want to implement part of the methods in
|
||||
* {@link AsyncClusterConnection}.
|
||||
*/
|
||||
public class DummyAsyncClusterConnection implements AsyncClusterConnection {
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncTableRegionLocator getRegionLocator(TableName tableName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearRegionLocationCache() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncTableBuilder<AdvancedScanResultConsumer> getTableBuilder(TableName tableName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName,
|
||||
ExecutorService pool) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncAdminBuilder getAdminBuilder() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncAdminBuilder getAdminBuilder(ExecutorService pool) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
|
||||
ExecutorService pool) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Hbck> getHbck() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Hbck getHbck(ServerName masterServer) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NonceGenerator getNonceGenerator() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RpcClient getRpcClient() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<FlushRegionResponse> flush(byte[] regionName,
|
||||
boolean writeFlushWALMarker) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Long> replay(TableName tableName, byte[] encodedRegionName, byte[] row,
|
||||
List<Entry> entries, int replicaId, int numRetries, long operationTimeoutNs) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
|
||||
boolean reload) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<String> prepareBulkLoad(TableName tableName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Boolean> bulkLoad(TableName tableName,
|
||||
List<Pair<byte[], String>> familyPaths, byte[] row, boolean assignSeqNum, Token<?> userToken,
|
||||
String bulkToken, boolean copyFiles) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> cleanupBulkLoad(TableName tableName, String bulkToken) {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
||||
/**
|
||||
* Can be overridden in UT if you only want to implement part of the methods in
|
||||
* {@link AsyncRegistry}.
|
||||
*/
|
||||
public class DummyAsyncRegistry implements AsyncRegistry {
|
||||
|
||||
public static final String REGISTRY_IMPL_CONF_KEY = AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY;
|
||||
|
||||
@Override
|
||||
public CompletableFuture<RegionLocations> getMetaRegionLocation() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<String> getClusterId() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Integer> getCurrentNrHRS() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<ServerName> getMasterAddress() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Integer> getMasterInfoPort() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
|
@ -0,0 +1,159 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
||||
/**
|
||||
* Can be overridden in UT if you only want to implement part of the methods in {@link AsyncTable}.
|
||||
*/
|
||||
public class DummyAsyncTable<C extends ScanResultConsumerBase> implements AsyncTable<C> {
|
||||
|
||||
@Override
|
||||
public TableName getName() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<TableDescriptor> getDescriptor() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncTableRegionLocator getRegionLocator() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getRpcTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getReadRpcTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getWriteRpcTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getOperationTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getScanTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Result> get(Get get) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> put(Put put) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> delete(Delete delete) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Result> append(Append append) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Result> increment(Increment increment) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> mutateRow(RowMutations mutation) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void scan(Scan scan, C consumer) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultScanner getScanner(Scan scan) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<List<Result>> scanAll(Scan scan) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CompletableFuture<Result>> get(List<Get> gets) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CompletableFuture<Void>> put(List<Put> puts) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CompletableFuture<Void>> delete(List<Delete> deletes) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
||||
ServiceCaller<S, R> callable, byte[] row) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
|
||||
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
|
||||
CoprocessorCallback<R> callback) {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -18,10 +18,12 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -44,15 +46,14 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
|||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
|
||||
import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
|
@ -471,40 +472,17 @@ public class TestReplicaWithCluster {
|
|||
final int numRows = 10;
|
||||
final byte[] qual = Bytes.toBytes("qual");
|
||||
final byte[] val = Bytes.toBytes("val");
|
||||
final List<Pair<byte[], String>> famPaths = new ArrayList<>();
|
||||
Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
for (HColumnDescriptor col : hdt.getColumnFamilies()) {
|
||||
Path hfile = new Path(dir, col.getNameAsString());
|
||||
TestHRegionServerBulkLoad.createHFile(HTU.getTestFileSystem(), hfile, col.getName(),
|
||||
qual, val, numRows);
|
||||
famPaths.add(new Pair<>(col.getName(), hfile.toString()));
|
||||
TestHRegionServerBulkLoad.createHFile(HTU.getTestFileSystem(), hfile, col.getName(), qual,
|
||||
val, numRows);
|
||||
family2Files.put(col.getName(), Collections.singletonList(hfile));
|
||||
}
|
||||
|
||||
// bulk load HFiles
|
||||
LOG.debug("Loading test data");
|
||||
final ClusterConnection conn = (ClusterConnection) HTU.getAdmin().getConnection();
|
||||
table = conn.getTable(hdt.getTableName());
|
||||
final String bulkToken =
|
||||
new SecureBulkLoadClient(HTU.getConfiguration(), table).prepareBulkLoad(conn);
|
||||
ClientServiceCallable<Void> callable = new ClientServiceCallable<Void>(conn,
|
||||
hdt.getTableName(), TestHRegionServerBulkLoad.rowkey(0),
|
||||
new RpcControllerFactory(HTU.getConfiguration()).newController(), HConstants.PRIORITY_UNSET) {
|
||||
@Override
|
||||
protected Void rpcCall() throws Exception {
|
||||
LOG.debug("Going to connect to server " + getLocation() + " for row "
|
||||
+ Bytes.toStringBinary(getRow()));
|
||||
SecureBulkLoadClient secureClient = null;
|
||||
byte[] regionName = getLocation().getRegion().getRegionName();
|
||||
try (Table table = conn.getTable(getTableName())) {
|
||||
secureClient = new SecureBulkLoadClient(HTU.getConfiguration(), table);
|
||||
secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
|
||||
true, null, bulkToken);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(HTU.getConfiguration());
|
||||
RpcRetryingCaller<Void> caller = factory.newCaller();
|
||||
caller.callWithRetries(callable, 10000);
|
||||
BulkLoadHFiles.create(HTU.getConfiguration()).bulkLoad(hdt.getTableName(), family2Files);
|
||||
|
||||
// verify we can read them from the primary
|
||||
LOG.debug("Verifying data load");
|
||||
|
|
|
@ -72,7 +72,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
|
|||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
|
@ -567,7 +567,7 @@ public class TestRegionObserverInterface {
|
|||
createHFile(util.getConfiguration(), fs, new Path(familyDir, Bytes.toString(A)), A, A);
|
||||
|
||||
// Bulk load
|
||||
new LoadIncrementalHFiles(conf).doBulkLoad(dir, util.getAdmin(), table, locator);
|
||||
BulkLoadHFiles.create(conf).bulkLoad(tableName, dir);
|
||||
|
||||
verifyMethodResult(SimpleRegionObserver.class,
|
||||
new String[] { "hadPreBulkLoadHFile", "hadPostBulkLoadHFile" }, tableName,
|
||||
|
|
|
@ -22,41 +22,38 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Objects;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Waiter.Predicate;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClientServiceCallable;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.junit.rules.TestName;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
|
||||
|
@ -378,39 +375,21 @@ public class SpaceQuotaHelperForTests {
|
|||
/**
|
||||
* Bulk-loads a number of files with a number of rows to the given table.
|
||||
*/
|
||||
ClientServiceCallable<Boolean> generateFileToLoad(
|
||||
TableName tn, int numFiles, int numRowsPerFile) throws Exception {
|
||||
Connection conn = testUtil.getConnection();
|
||||
Map<byte[], List<Path>> generateFileToLoad(TableName tn, int numFiles, int numRowsPerFile)
|
||||
throws Exception {
|
||||
FileSystem fs = testUtil.getTestFileSystem();
|
||||
Configuration conf = testUtil.getConfiguration();
|
||||
Path baseDir = new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files");
|
||||
fs.mkdirs(baseDir);
|
||||
final List<Pair<byte[], String>> famPaths = new ArrayList<>();
|
||||
List<Path> hfiles = new ArrayList<>();
|
||||
for (int i = 1; i <= numFiles; i++) {
|
||||
Path hfile = new Path(baseDir, "file" + i);
|
||||
TestHRegionServerBulkLoad.createHFile(
|
||||
fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("my"),
|
||||
Bytes.toBytes("file"), numRowsPerFile);
|
||||
famPaths.add(new Pair<>(Bytes.toBytes(SpaceQuotaHelperForTests.F1), hfile.toString()));
|
||||
TestHRegionServerBulkLoad.createHFile(fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1),
|
||||
Bytes.toBytes("my"), Bytes.toBytes("file"), numRowsPerFile);
|
||||
hfiles.add(hfile);
|
||||
}
|
||||
|
||||
// bulk load HFiles
|
||||
Table table = conn.getTable(tn);
|
||||
final String bulkToken = new SecureBulkLoadClient(conf, table).prepareBulkLoad(conn);
|
||||
return new ClientServiceCallable<Boolean>(
|
||||
conn, tn, Bytes.toBytes("row"), new RpcControllerFactory(conf).newController(),
|
||||
HConstants.PRIORITY_UNSET) {
|
||||
@Override
|
||||
public Boolean rpcCall() throws Exception {
|
||||
SecureBulkLoadClient secureClient = null;
|
||||
byte[] regionName = getLocation().getRegion().getRegionName();
|
||||
try (Table table = conn.getTable(getTableName())) {
|
||||
secureClient = new SecureBulkLoadClient(conf, table);
|
||||
return secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
|
||||
true, null, bulkToken);
|
||||
}
|
||||
}
|
||||
};
|
||||
Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
family2Files.put(Bytes.toBytes(SpaceQuotaHelperForTests.F1), hfiles);
|
||||
return family2Files;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,12 +17,13 @@
|
|||
package org.apache.hadoop.hbase.quotas;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -31,11 +32,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClientServiceCallable;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCaller;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.client.SnapshotType;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.quotas.SpaceQuotaHelperForTests.SpaceQuotaSnapshotPredicate;
|
||||
|
@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -214,7 +213,7 @@ public class TestLowLatencySpaceQuotas {
|
|||
tn, SpaceQuotaHelperForTests.ONE_GIGABYTE, SpaceViolationPolicy.NO_INSERTS);
|
||||
admin.setQuota(settings);
|
||||
|
||||
ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tn, 3, 550);
|
||||
Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tn, 3, 550);
|
||||
// Make sure the files are about as long as we expect
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
FileStatus[] files = fs.listStatus(
|
||||
|
@ -228,13 +227,13 @@ public class TestLowLatencySpaceQuotas {
|
|||
totalSize += file.getLen();
|
||||
}
|
||||
|
||||
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
|
||||
RpcRetryingCaller<Boolean> caller = factory.<Boolean> newCaller();
|
||||
assertTrue("The bulk load failed", caller.callWithRetries(callable, Integer.MAX_VALUE));
|
||||
assertFalse("The bulk load failed",
|
||||
BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tn, family2Files).isEmpty());
|
||||
|
||||
final long finalTotalSize = totalSize;
|
||||
TEST_UTIL.waitFor(30 * 1000, 500, new SpaceQuotaSnapshotPredicate(conn, tn) {
|
||||
@Override boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
|
||||
@Override
|
||||
boolean evaluate(SpaceQuotaSnapshot snapshot) throws Exception {
|
||||
return snapshot.getUsage() >= finalTotalSize;
|
||||
}
|
||||
});
|
||||
|
|
|
@ -17,13 +17,17 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.quotas;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -38,7 +42,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.ClientServiceCallable;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
|
@ -47,8 +50,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCaller;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
|
@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.quotas.policies.DefaultViolationPolicyEnforcement
|
|||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.security.AccessDeniedException;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -237,19 +239,18 @@ public class TestSpaceQuotas {
|
|||
@Test
|
||||
public void testNoBulkLoadsWithNoWrites() throws Exception {
|
||||
Put p = new Put(Bytes.toBytes("to_reject"));
|
||||
p.addColumn(
|
||||
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
|
||||
p.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
|
||||
Bytes.toBytes("reject"));
|
||||
TableName tableName = writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
|
||||
|
||||
// The table is now in violation. Try to do a bulk load
|
||||
ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tableName, 1, 50);
|
||||
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
|
||||
RpcRetryingCaller<Boolean> caller = factory.newCaller();
|
||||
Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tableName, 1, 50);
|
||||
try {
|
||||
caller.callWithRetries(callable, Integer.MAX_VALUE);
|
||||
BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tableName, family2Files);
|
||||
fail("Expected the bulk load call to fail!");
|
||||
} catch (SpaceLimitingException e) {
|
||||
} catch (IOException e) {
|
||||
// Pass
|
||||
assertThat(e.getCause(), instanceOf(SpaceLimitingException.class));
|
||||
LOG.trace("Caught expected exception", e);
|
||||
}
|
||||
}
|
||||
|
@ -293,7 +294,7 @@ public class TestSpaceQuotas {
|
|||
enforcement instanceof DefaultViolationPolicyEnforcement);
|
||||
|
||||
// Should generate two files, each of which is over 25KB each
|
||||
ClientServiceCallable<Boolean> callable = helper.generateFileToLoad(tn, 2, 525);
|
||||
Map<byte[], List<Path>> family2Files = helper.generateFileToLoad(tn, 2, 525);
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
FileStatus[] files = fs.listStatus(
|
||||
new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files"));
|
||||
|
@ -305,13 +306,12 @@ public class TestSpaceQuotas {
|
|||
LOG.debug(file.getPath() + " -> " + file.getLen() +"B");
|
||||
}
|
||||
|
||||
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
|
||||
RpcRetryingCaller<Boolean> caller = factory.newCaller();
|
||||
try {
|
||||
caller.callWithRetries(callable, Integer.MAX_VALUE);
|
||||
BulkLoadHFiles.create(TEST_UTIL.getConfiguration()).bulkLoad(tn, family2Files);
|
||||
fail("Expected the bulk load call to fail!");
|
||||
} catch (SpaceLimitingException e) {
|
||||
} catch (IOException e) {
|
||||
// Pass
|
||||
assertThat(e.getCause(), instanceOf(SpaceLimitingException.class));
|
||||
LOG.trace("Caught expected exception", e);
|
||||
}
|
||||
// Verify that we have no data in the table because neither file should have been
|
||||
|
|
|
@ -25,9 +25,11 @@ import java.io.IOException;
|
|||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -53,7 +55,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
|||
import org.apache.hadoop.hbase.client.RpcRetryingCaller;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
|
@ -71,8 +72,8 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
@ -204,59 +205,37 @@ public class TestHRegionServerBulkLoad {
|
|||
// create HFiles for different column families
|
||||
FileSystem fs = UTIL.getTestFileSystem();
|
||||
byte[] val = Bytes.toBytes(String.format("%010d", iteration));
|
||||
final List<Pair<byte[], String>> famPaths = new ArrayList<>(NUM_CFS);
|
||||
Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
for (int i = 0; i < NUM_CFS; i++) {
|
||||
Path hfile = new Path(dir, family(i));
|
||||
byte[] fam = Bytes.toBytes(family(i));
|
||||
createHFile(fs, hfile, fam, QUAL, val, 1000);
|
||||
famPaths.add(new Pair<>(fam, hfile.toString()));
|
||||
family2Files.put(fam, Collections.singletonList(hfile));
|
||||
}
|
||||
|
||||
// bulk load HFiles
|
||||
final ClusterConnection conn = (ClusterConnection)UTIL.getConnection();
|
||||
Table table = conn.getTable(tableName);
|
||||
final String bulkToken = new SecureBulkLoadClient(UTIL.getConfiguration(), table).
|
||||
prepareBulkLoad(conn);
|
||||
ClientServiceCallable<Void> callable = new ClientServiceCallable<Void>(conn,
|
||||
tableName, Bytes.toBytes("aaa"),
|
||||
new RpcControllerFactory(UTIL.getConfiguration()).newController(), HConstants.PRIORITY_UNSET) {
|
||||
@Override
|
||||
public Void rpcCall() throws Exception {
|
||||
LOG.debug("Going to connect to server " + getLocation() + " for row "
|
||||
+ Bytes.toStringBinary(getRow()));
|
||||
SecureBulkLoadClient secureClient = null;
|
||||
byte[] regionName = getLocation().getRegion().getRegionName();
|
||||
try (Table table = conn.getTable(getTableName())) {
|
||||
secureClient = new SecureBulkLoadClient(UTIL.getConfiguration(), table);
|
||||
secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
|
||||
true, null, bulkToken);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf);
|
||||
RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
|
||||
caller.callWithRetries(callable, Integer.MAX_VALUE);
|
||||
|
||||
BulkLoadHFiles.create(UTIL.getConfiguration()).bulkLoad(tableName, family2Files);
|
||||
// Periodically do compaction to reduce the number of open file handles.
|
||||
if (numBulkLoads.get() % 5 == 0) {
|
||||
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf);
|
||||
RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
|
||||
// 5 * 50 = 250 open file handles!
|
||||
callable = new ClientServiceCallable<Void>(conn,
|
||||
tableName, Bytes.toBytes("aaa"),
|
||||
new RpcControllerFactory(UTIL.getConfiguration()).newController(), HConstants.PRIORITY_UNSET) {
|
||||
@Override
|
||||
protected Void rpcCall() throws Exception {
|
||||
LOG.debug("compacting " + getLocation() + " for row "
|
||||
+ Bytes.toStringBinary(getRow()));
|
||||
AdminProtos.AdminService.BlockingInterface server =
|
||||
conn.getAdmin(getLocation().getServerName());
|
||||
CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
|
||||
ClientServiceCallable<Void> callable =
|
||||
new ClientServiceCallable<Void>(UTIL.getConnection(), tableName, Bytes.toBytes("aaa"),
|
||||
new RpcControllerFactory(UTIL.getConfiguration()).newController(),
|
||||
HConstants.PRIORITY_UNSET) {
|
||||
@Override
|
||||
protected Void rpcCall() throws Exception {
|
||||
LOG.debug(
|
||||
"compacting " + getLocation() + " for row " + Bytes.toStringBinary(getRow()));
|
||||
AdminProtos.AdminService.BlockingInterface server =
|
||||
((ClusterConnection) UTIL.getConnection()).getAdmin(getLocation().getServerName());
|
||||
CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
|
||||
getLocation().getRegion().getRegionName(), true, null);
|
||||
server.compactRegion(null, request);
|
||||
numCompactions.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
};
|
||||
server.compactRegion(null, request);
|
||||
numCompactions.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
};
|
||||
caller.callWithRetries(callable, Integer.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
|
@ -94,10 +93,7 @@ public class TestScannerWithBulkload {
|
|||
false);
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
|
||||
final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
|
||||
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
bulkload.doBulkLoad(hfilePath, admin, table, locator);
|
||||
}
|
||||
BulkLoadHFiles.create(conf).bulkLoad(tableName, hfilePath);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
Result result = scanner.next();
|
||||
result = scanAfterBulkLoad(scanner, result, "version2");
|
||||
|
@ -233,7 +229,7 @@ public class TestScannerWithBulkload {
|
|||
"/temp/testBulkLoadWithParallelScan/col/file", false);
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
|
||||
final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
|
||||
final BulkLoadHFiles bulkload = BulkLoadHFiles.create(conf);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
Result result = scanner.next();
|
||||
// Create a scanner and then do bulk load
|
||||
|
@ -246,9 +242,7 @@ public class TestScannerWithBulkload {
|
|||
put1.add(new KeyValue(Bytes.toBytes("row5"), Bytes.toBytes("col"), Bytes.toBytes("q"), l,
|
||||
Bytes.toBytes("version0")));
|
||||
table.put(put1);
|
||||
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
bulkload.doBulkLoad(hfilePath, admin, table, locator);
|
||||
}
|
||||
bulkload.bulkLoad(tableName, hfilePath);
|
||||
latch.countDown();
|
||||
} catch (TableNotFoundException e) {
|
||||
} catch (IOException e) {
|
||||
|
@ -276,10 +270,7 @@ public class TestScannerWithBulkload {
|
|||
"/temp/testBulkLoadNativeHFile/col/file", true);
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
|
||||
final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
|
||||
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
bulkload.doBulkLoad(hfilePath, admin, table, locator);
|
||||
}
|
||||
BulkLoadHFiles.create(conf).bulkLoad(tableName, hfilePath);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
Result result = scanner.next();
|
||||
// We had 'version0', 'version1' for 'row1,col:q' in the table.
|
||||
|
|
|
@ -21,10 +21,8 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.Deque;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -32,8 +30,8 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
|
|||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.AsyncClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
|
@ -47,7 +45,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -59,6 +57,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
|
||||
|
||||
|
||||
|
@ -178,7 +177,7 @@ public class TestSecureBulkLoadManager {
|
|||
|
||||
/**
|
||||
* A trick is used to make sure server-side failures( if any ) not being covered up by a client
|
||||
* retry. Since LoadIncrementalHFiles.doBulkLoad keeps performing bulkload calls as long as the
|
||||
* retry. Since BulkLoadHFilesTool.bulkLoad keeps performing bulkload calls as long as the
|
||||
* HFile queue is not empty, while server-side exceptions in the doAs block do not lead
|
||||
* to a client exception, a bulkload will always succeed in this case by default, thus client
|
||||
* will never be aware that failures have ever happened . To avoid this kind of retry ,
|
||||
|
@ -187,23 +186,23 @@ public class TestSecureBulkLoadManager {
|
|||
* once, and server-side failures, if any ,can be checked via data.
|
||||
*/
|
||||
class MyExceptionToAvoidRetry extends DoNotRetryIOException {
|
||||
|
||||
private static final long serialVersionUID = -6802760664998771151L;
|
||||
}
|
||||
|
||||
private void doBulkloadWithoutRetry(Path dir) throws Exception {
|
||||
Connection connection = testUtil.getConnection();
|
||||
LoadIncrementalHFiles h = new LoadIncrementalHFiles(conf) {
|
||||
BulkLoadHFilesTool h = new BulkLoadHFilesTool(conf) {
|
||||
|
||||
@Override
|
||||
protected void bulkLoadPhase(final Table htable, final Connection conn,
|
||||
ExecutorService pool, Deque<LoadQueueItem> queue,
|
||||
final Multimap<ByteBuffer, LoadQueueItem> regionGroups, boolean copyFile,
|
||||
Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
|
||||
super.bulkLoadPhase(htable, conn, pool, queue, regionGroups, copyFile, item2RegionMap);
|
||||
protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
|
||||
Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
|
||||
boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
|
||||
super.bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, item2RegionMap);
|
||||
throw new MyExceptionToAvoidRetry(); // throw exception to avoid retry
|
||||
}
|
||||
};
|
||||
try {
|
||||
h.doBulkLoad(dir, testUtil.getAdmin(), connection.getTable(TABLE),
|
||||
connection.getRegionLocator(TABLE));
|
||||
h.bulkLoad(TABLE, dir);
|
||||
Assert.fail("MyExceptionToAvoidRetry is expected");
|
||||
} catch (MyExceptionToAvoidRetry e) { //expected
|
||||
}
|
||||
|
|
|
@ -67,7 +67,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
|||
import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.HFileTestUtil;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
|
@ -622,9 +622,7 @@ public class TestMasterReplication {
|
|||
|
||||
Table source = tables[masterNumber];
|
||||
final TableName tableName = source.getName();
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
|
||||
String[] args = { dir.toString(), tableName.toString() };
|
||||
loader.run(args);
|
||||
BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(tableName, dir);
|
||||
|
||||
if (toValidate) {
|
||||
for (int slaveClusterNumber : slaveNumbers) {
|
||||
|
|
|
@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -275,7 +275,7 @@ public class TestReplicationSink {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRethrowRetriesExhaustedWithDetailsException() throws Exception {
|
||||
public void testRethrowRetriesExhaustedException() throws Exception {
|
||||
TableName notExistTable = TableName.valueOf("notExistTable");
|
||||
List<WALEntry> entries = new ArrayList<>();
|
||||
List<Cell> cells = new ArrayList<>();
|
||||
|
@ -300,7 +300,7 @@ public class TestReplicationSink {
|
|||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
Assert.fail("Should re-throw RetriesExhaustedWithDetailsException.");
|
||||
} catch (RetriesExhaustedWithDetailsException e) {
|
||||
} catch (RetriesExhaustedException e) {
|
||||
} finally {
|
||||
admin.enableTable(TABLE_NAME1);
|
||||
}
|
||||
|
|
|
@ -21,11 +21,10 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.SocketAddress;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -38,27 +37,15 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutatorParams;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
|
||||
import org.apache.hadoop.hbase.client.AsyncClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.AsyncConnection;
|
||||
import org.apache.hadoop.hbase.client.AsyncTable;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.DummyAsyncClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.DummyAsyncRegistry;
|
||||
import org.apache.hadoop.hbase.client.DummyAsyncTable;
|
||||
import org.apache.hadoop.hbase.client.Row;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -78,15 +65,16 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
|||
/**
|
||||
* Simple test of sink-side wal entry filter facility.
|
||||
*/
|
||||
@Category({ReplicationTests.class, SmallTests.class})
|
||||
@Category({ ReplicationTests.class, SmallTests.class })
|
||||
public class TestWALEntrySinkFilter {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestWALEntrySinkFilter.class);
|
||||
HBaseClassTestRule.forClass(TestWALEntrySinkFilter.class);
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestReplicationSink.class);
|
||||
@Rule public TestName name = new TestName();
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
static final int BOUNDARY = 5;
|
||||
static final AtomicInteger UNFILTERED = new AtomicInteger();
|
||||
static final AtomicInteger FILTERED = new AtomicInteger();
|
||||
|
@ -110,55 +98,48 @@ public class TestWALEntrySinkFilter {
|
|||
};
|
||||
|
||||
/**
|
||||
* Test filter.
|
||||
* Filter will filter out any write time that is <= 5 (BOUNDARY). We count how many items we
|
||||
* filter out and we count how many cells make it through for distribution way down below in the
|
||||
* Table#batch implementation. Puts in place a custom DevNullConnection so we can insert our
|
||||
* counting Table.
|
||||
* Test filter. Filter will filter out any write time that is <= 5 (BOUNDARY). We count how many
|
||||
* items we filter out and we count how many cells make it through for distribution way down below
|
||||
* in the Table#batch implementation. Puts in place a custom DevNullConnection so we can insert
|
||||
* our counting Table.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testWALEntryFilter() throws IOException {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
// Make it so our filter is instantiated on construction of ReplicationSink.
|
||||
conf.setClass(DummyAsyncRegistry.REGISTRY_IMPL_CONF_KEY, DevNullAsyncRegistry.class,
|
||||
DummyAsyncRegistry.class);
|
||||
conf.setClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY,
|
||||
IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
|
||||
conf.setClass("hbase.client.connection.impl", DevNullConnection.class,
|
||||
Connection.class);
|
||||
IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
|
||||
conf.setClass(ClusterConnectionFactory.HBASE_SERVER_CLUSTER_CONNECTION_IMPL,
|
||||
DevNullAsyncClusterConnection.class, AsyncClusterConnection.class);
|
||||
ReplicationSink sink = new ReplicationSink(conf, STOPPABLE);
|
||||
// Create some dumb walentries.
|
||||
List< org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry > entries =
|
||||
new ArrayList<>();
|
||||
List<AdminProtos.WALEntry> entries = new ArrayList<>();
|
||||
AdminProtos.WALEntry.Builder entryBuilder = AdminProtos.WALEntry.newBuilder();
|
||||
// Need a tablename.
|
||||
ByteString tableName =
|
||||
ByteString.copyFromUtf8(TableName.valueOf(this.name.getMethodName()).toString());
|
||||
ByteString.copyFromUtf8(TableName.valueOf(this.name.getMethodName()).toString());
|
||||
// Add WALEdit Cells to Cells List. The way edits arrive at the sink is with protos
|
||||
// describing the edit with all Cells from all edits aggregated in a single CellScanner.
|
||||
final List<Cell> cells = new ArrayList<>();
|
||||
int count = BOUNDARY * 2;
|
||||
for(int i = 0; i < count; i++) {
|
||||
byte [] bytes = Bytes.toBytes(i);
|
||||
for (int i = 0; i < count; i++) {
|
||||
byte[] bytes = Bytes.toBytes(i);
|
||||
// Create a wal entry. Everything is set to the current index as bytes or int/long.
|
||||
entryBuilder.clear();
|
||||
entryBuilder.setKey(entryBuilder.getKeyBuilder().
|
||||
setLogSequenceNumber(i).
|
||||
setEncodedRegionName(ByteString.copyFrom(bytes)).
|
||||
setWriteTime(i).
|
||||
setTableName(tableName).build());
|
||||
entryBuilder.setKey(entryBuilder.getKeyBuilder().setLogSequenceNumber(i)
|
||||
.setEncodedRegionName(ByteString.copyFrom(bytes)).setWriteTime(i).setTableName(tableName)
|
||||
.build());
|
||||
// Lets have one Cell associated with each WALEdit.
|
||||
entryBuilder.setAssociatedCellCount(1);
|
||||
entries.add(entryBuilder.build());
|
||||
// We need to add a Cell per WALEdit to the cells array.
|
||||
CellBuilder cellBuilder = CellBuilderFactory.create(CellBuilderType.DEEP_COPY);
|
||||
// Make cells whose row, family, cell, value, and ts are == 'i'.
|
||||
Cell cell = cellBuilder.
|
||||
setRow(bytes).
|
||||
setFamily(bytes).
|
||||
setQualifier(bytes).
|
||||
setType(Cell.Type.Put).
|
||||
setTimestamp(i).
|
||||
setValue(bytes).build();
|
||||
Cell cell = cellBuilder.setRow(bytes).setFamily(bytes).setQualifier(bytes)
|
||||
.setType(Cell.Type.Put).setTimestamp(i).setValue(bytes).build();
|
||||
cells.add(cell);
|
||||
}
|
||||
// Now wrap our cells array in a CellScanner that we can pass in to replicateEntries. It has
|
||||
|
@ -189,11 +170,13 @@ public class TestWALEntrySinkFilter {
|
|||
/**
|
||||
* Simple filter that will filter out any entry wholse writeTime is <= 5.
|
||||
*/
|
||||
public static class IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl implements WALEntrySinkFilter {
|
||||
public IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl() {}
|
||||
public static class IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl
|
||||
implements WALEntrySinkFilter {
|
||||
public IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(Connection connection) {
|
||||
public void init(AsyncConnection conn) {
|
||||
// Do nothing.
|
||||
}
|
||||
|
||||
|
@ -207,266 +190,48 @@ public class TestWALEntrySinkFilter {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A DevNull Connection whose only purpose is checking what edits made it through. See down in
|
||||
* {@link Table#batch(List, Object[])}.
|
||||
*/
|
||||
public static class DevNullConnection implements Connection {
|
||||
private final Configuration configuration;
|
||||
public static class DevNullAsyncRegistry extends DummyAsyncRegistry {
|
||||
|
||||
DevNullConnection(Configuration configuration, ExecutorService es, User user) {
|
||||
this.configuration = configuration;
|
||||
public DevNullAsyncRegistry(Configuration conf) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
public CompletableFuture<String> getClusterId() {
|
||||
return CompletableFuture.completedFuture("test");
|
||||
}
|
||||
}
|
||||
|
||||
public static class DevNullAsyncClusterConnection extends DummyAsyncClusterConnection {
|
||||
|
||||
private final Configuration conf;
|
||||
|
||||
public DevNullAsyncClusterConnection(Configuration conf, Object registry, String clusterId,
|
||||
SocketAddress localAddress, User user) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return this.configuration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RegionLocator getRegionLocator(TableName tableName) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Admin getAdmin() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableBuilder getTableBuilder(final TableName tableName, ExecutorService pool) {
|
||||
return new TableBuilder() {
|
||||
@Override
|
||||
public TableBuilder setOperationTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
public AsyncTable<AdvancedScanResultConsumer> getTable(TableName tableName) {
|
||||
return new DummyAsyncTable<AdvancedScanResultConsumer>() {
|
||||
|
||||
@Override
|
||||
public TableBuilder setRpcTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableBuilder setReadRpcTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableBuilder setWriteRpcTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Table build() {
|
||||
return new Table() {
|
||||
@Override
|
||||
public TableName getName() {
|
||||
return tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return configuration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor getDescriptor() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean exists(Get get) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean[] exists(List<Get> gets) throws IOException {
|
||||
return new boolean[0];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void batch(List<? extends Row> actions, Object[] results) throws IOException, InterruptedException {
|
||||
for (Row action: actions) {
|
||||
// Row is the index of the loop above where we make WALEntry and Cells.
|
||||
int row = Bytes.toInt(action.getRow());
|
||||
assertTrue("" + row, row> BOUNDARY);
|
||||
UNFILTERED.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <R> void batchCallback(List<? extends Row> actions, Object[] results, Batch.Callback<R> callback) throws IOException, InterruptedException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result get(Get get) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result[] get(List<Get> gets) throws IOException {
|
||||
return new Result[0];
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultScanner getScanner(Scan scan) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultScanner getScanner(byte[] family) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(Put put) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(List<Put> puts) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(Delete delete) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(List<Delete> deletes) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void mutateRow(RowMutations rm) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result append(Append append) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result increment(Increment increment) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, Durability durability) throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoprocessorRpcChannel coprocessorService(byte[] row) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T extends com.google.protobuf.Service, R> Map<byte[], R> coprocessorService(Class<T> service, byte[] startKey, byte[] endKey, Batch.Call<T, R> callable) throws com.google.protobuf.ServiceException, Throwable {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T extends com.google.protobuf.Service, R> void coprocessorService(Class<T> service, byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback) throws com.google.protobuf.ServiceException, Throwable {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public <R extends com.google.protobuf.Message> Map<byte[], R> batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor, com.google.protobuf.Message request, byte[] startKey, byte[] endKey, R responsePrototype) throws com.google.protobuf.ServiceException, Throwable {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <R extends com.google.protobuf.Message> void batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor, com.google.protobuf.Message request, byte[] startKey, byte[] endKey, R responsePrototype, Batch.Callback<R> callback) throws com.google.protobuf.ServiceException, Throwable {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getRpcTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getReadRpcTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getWriteRpcTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getOperationTimeout(TimeUnit unit) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RegionLocator getRegionLocator() throws IOException {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
public <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
|
||||
List<T> list = new ArrayList<>(actions.size());
|
||||
for (Row action : actions) {
|
||||
// Row is the index of the loop above where we make WALEntry and Cells.
|
||||
int row = Bytes.toInt(action.getRow());
|
||||
assertTrue("" + row, row > BOUNDARY);
|
||||
UNFILTERED.incrementAndGet();
|
||||
list.add(null);
|
||||
}
|
||||
return CompletableFuture.completedFuture(list);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearRegionLocationCache() {
|
||||
public Configuration getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -125,7 +125,7 @@ import org.apache.hadoop.hbase.security.User;
|
|||
import org.apache.hadoop.hbase.security.access.Permission.Action;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SecurityTests;
|
||||
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -1112,14 +1112,8 @@ public class TestAccessController extends SecureTestUtil {
|
|||
}
|
||||
|
||||
private void bulkLoadHFile(TableName tableName) throws Exception {
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Admin admin = conn.getAdmin();
|
||||
RegionLocator locator = conn.getRegionLocator(tableName);
|
||||
Table table = conn.getTable(tableName)) {
|
||||
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
|
||||
loader.doBulkLoad(loadPath, admin, table, locator);
|
||||
}
|
||||
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
|
||||
BulkLoadHFiles.create(conf).bulkLoad(tableName, loadPath);
|
||||
}
|
||||
|
||||
private static void setPermission(FileSystem fs, Path dir, FsPermission perm)
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.tool;
|
||||
|
||||
import static org.apache.hadoop.hbase.HBaseTestingUtility.countRows;
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -72,11 +73,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
|||
* faster than the full MR cluster tests in TestHFileOutputFormat
|
||||
*/
|
||||
@Category({ MiscTests.class, LargeTests.class })
|
||||
public class TestLoadIncrementalHFiles {
|
||||
public class TestBulkLoadHFiles {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestLoadIncrementalHFiles.class);
|
||||
HBaseClassTestRule.forClass(TestBulkLoadHFiles.class);
|
||||
|
||||
@Rule
|
||||
public TestName tn = new TestName();
|
||||
|
@ -89,14 +90,14 @@ public class TestLoadIncrementalHFiles {
|
|||
static final int MAX_FILES_PER_REGION_PER_FAMILY = 4;
|
||||
|
||||
private static final byte[][] SPLIT_KEYS =
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ppp") };
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ppp") };
|
||||
|
||||
static HBaseTestingUtility util = new HBaseTestingUtility();
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
|
||||
util.getConfiguration().setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
|
||||
util.getConfiguration().setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
|
||||
MAX_FILES_PER_REGION_PER_FAMILY);
|
||||
// change default behavior so that tag values are returned with normal rpcs
|
||||
util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
|
||||
|
@ -119,7 +120,7 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testSimpleLoadWithMap() throws Exception {
|
||||
runTest("testSimpleLoadWithMap", BloomType.NONE,
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
|
||||
true);
|
||||
}
|
||||
|
||||
|
@ -130,16 +131,16 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testSimpleLoad() throws Exception {
|
||||
runTest("testSimpleLoad", BloomType.NONE,
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, });
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, });
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleLoadWithFileCopy() throws Exception {
|
||||
String testName = tn.getMethodName();
|
||||
final byte[] TABLE_NAME = Bytes.toBytes("mytable_" + testName);
|
||||
runTest(testName, buildHTD(TableName.valueOf(TABLE_NAME), BloomType.NONE),
|
||||
false, null, new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
|
||||
runTest(testName, buildHTD(TableName.valueOf(TABLE_NAME), BloomType.NONE), false, null,
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, },
|
||||
false, true, 2);
|
||||
}
|
||||
|
||||
|
@ -150,7 +151,7 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testRegionCrossingLoad() throws Exception {
|
||||
runTest("testRegionCrossingLoad", BloomType.NONE,
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -160,7 +161,7 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testRegionCrossingRowBloom() throws Exception {
|
||||
runTest("testRegionCrossingLoadRowBloom", BloomType.ROW,
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -170,7 +171,7 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testRegionCrossingRowColBloom() throws Exception {
|
||||
runTest("testRegionCrossingLoadRowColBloom", BloomType.ROWCOL,
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -181,9 +182,9 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testSimpleHFileSplit() throws Exception {
|
||||
runTest("testHFileSplit", BloomType.NONE,
|
||||
new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("fff"), Bytes.toBytes("jjj"),
|
||||
Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
|
||||
Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("lll") },
|
||||
new byte[][] { Bytes.toBytes("mmm"), Bytes.toBytes("zzz") }, });
|
||||
new byte[][] { Bytes.toBytes("mmm"), Bytes.toBytes("zzz") }, });
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -217,27 +218,27 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testSplitALot() throws Exception {
|
||||
runTest("testSplitALot", BloomType.NONE,
|
||||
new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("bbb"), Bytes.toBytes("ccc"),
|
||||
Bytes.toBytes("ddd"), Bytes.toBytes("eee"), Bytes.toBytes("fff"), Bytes.toBytes("ggg"),
|
||||
Bytes.toBytes("hhh"), Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
|
||||
Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
|
||||
Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"), Bytes.toBytes("uuu"),
|
||||
Bytes.toBytes("vvv"), Bytes.toBytes("zzz"), },
|
||||
Bytes.toBytes("ddd"), Bytes.toBytes("eee"), Bytes.toBytes("fff"), Bytes.toBytes("ggg"),
|
||||
Bytes.toBytes("hhh"), Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
|
||||
Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
|
||||
Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"), Bytes.toBytes("uuu"),
|
||||
Bytes.toBytes("vvv"), Bytes.toBytes("zzz"), },
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("zzz") }, });
|
||||
}
|
||||
|
||||
private void testRegionCrossingHFileSplit(BloomType bloomType) throws Exception {
|
||||
runTest("testHFileSplit" + bloomType + "Bloom", bloomType,
|
||||
new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("fff"), Bytes.toBytes("jjj"),
|
||||
Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
|
||||
Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), },
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("eee") },
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
new byte[][] { Bytes.toBytes("fff"), Bytes.toBytes("zzz") }, });
|
||||
}
|
||||
|
||||
private TableDescriptor buildHTD(TableName tableName, BloomType bloomType) {
|
||||
return TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setColumnFamily(
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBloomFilterType(bloomType).build())
|
||||
.build();
|
||||
.setColumnFamily(
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBloomFilterType(bloomType).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
private void runTest(String testName, BloomType bloomType, byte[][][] hfileRanges)
|
||||
|
@ -265,28 +266,24 @@ public class TestLoadIncrementalHFiles {
|
|||
runTest(testName, TABLE_WITHOUT_NS, bloomType, preCreateTable, tableSplitKeys, hfileRanges,
|
||||
useMap, 2);
|
||||
|
||||
|
||||
/* Run the test bulkloading the table from a depth of 3
|
||||
directory structure is now
|
||||
baseDirectory
|
||||
-- regionDir
|
||||
-- familyDir
|
||||
-- storeFileDir
|
||||
*/
|
||||
/*
|
||||
* Run the test bulkloading the table from a depth of 3 directory structure is now baseDirectory
|
||||
* -- regionDir -- familyDir -- storeFileDir
|
||||
*/
|
||||
if (preCreateTable) {
|
||||
runTest(testName + 2, TABLE_WITHOUT_NS, bloomType, true, tableSplitKeys, hfileRanges,
|
||||
false, 3);
|
||||
runTest(testName + 2, TABLE_WITHOUT_NS, bloomType, true, tableSplitKeys, hfileRanges, false,
|
||||
3);
|
||||
}
|
||||
|
||||
// Run the test bulkloading the table to the specified namespace
|
||||
final TableName TABLE_WITH_NS = TableName.valueOf(Bytes.toBytes(NAMESPACE), TABLE_NAME);
|
||||
runTest(testName, TABLE_WITH_NS, bloomType, preCreateTable, tableSplitKeys, hfileRanges,
|
||||
useMap, 2);
|
||||
runTest(testName, TABLE_WITH_NS, bloomType, preCreateTable, tableSplitKeys, hfileRanges, useMap,
|
||||
2);
|
||||
}
|
||||
|
||||
private void runTest(String testName, TableName tableName, BloomType bloomType,
|
||||
boolean preCreateTable, byte[][] tableSplitKeys, byte[][][] hfileRanges,
|
||||
boolean useMap, int depth) throws Exception {
|
||||
boolean preCreateTable, byte[][] tableSplitKeys, byte[][][] hfileRanges, boolean useMap,
|
||||
int depth) throws Exception {
|
||||
TableDescriptor htd = buildHTD(tableName, bloomType);
|
||||
runTest(testName, htd, preCreateTable, tableSplitKeys, hfileRanges, useMap, false, depth);
|
||||
}
|
||||
|
@ -296,7 +293,7 @@ public class TestLoadIncrementalHFiles {
|
|||
byte[][][] hfileRanges, boolean useMap, boolean deleteFile, boolean copyFiles,
|
||||
int initRowCount, int factor) throws Exception {
|
||||
return loadHFiles(testName, htd, util, fam, qual, preCreateTable, tableSplitKeys, hfileRanges,
|
||||
useMap, deleteFile, copyFiles, initRowCount, factor, 2);
|
||||
useMap, deleteFile, copyFiles, initRowCount, factor, 2);
|
||||
}
|
||||
|
||||
public static int loadHFiles(String testName, TableDescriptor htd, HBaseTestingUtility util,
|
||||
|
@ -343,7 +340,7 @@ public class TestLoadIncrementalHFiles {
|
|||
|
||||
Configuration conf = util.getConfiguration();
|
||||
if (copyFiles) {
|
||||
conf.setBoolean(LoadIncrementalHFiles.ALWAYS_COPY_FILES, true);
|
||||
conf.setBoolean(BulkLoadHFiles.ALWAYS_COPY_FILES, true);
|
||||
}
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf);
|
||||
List<String> args = Lists.newArrayList(baseDirectory.toString(), tableName.toString());
|
||||
|
@ -374,26 +371,23 @@ public class TestLoadIncrementalHFiles {
|
|||
}
|
||||
}
|
||||
|
||||
Table table = util.getConnection().getTable(tableName);
|
||||
try {
|
||||
assertEquals(initRowCount + expectedRows, util.countRows(table));
|
||||
} finally {
|
||||
table.close();
|
||||
try (Table table = util.getConnection().getTable(tableName)) {
|
||||
assertEquals(initRowCount + expectedRows, countRows(table));
|
||||
}
|
||||
|
||||
return expectedRows;
|
||||
}
|
||||
|
||||
private void runTest(String testName, TableDescriptor htd,
|
||||
boolean preCreateTable, byte[][] tableSplitKeys, byte[][][] hfileRanges, boolean useMap,
|
||||
boolean copyFiles, int depth) throws Exception {
|
||||
private void runTest(String testName, TableDescriptor htd, boolean preCreateTable,
|
||||
byte[][] tableSplitKeys, byte[][][] hfileRanges, boolean useMap, boolean copyFiles, int depth)
|
||||
throws Exception {
|
||||
loadHFiles(testName, htd, util, FAMILY, QUALIFIER, preCreateTable, tableSplitKeys, hfileRanges,
|
||||
useMap, true, copyFiles, 0, 1000, depth);
|
||||
|
||||
final TableName tableName = htd.getTableName();
|
||||
// verify staging folder has been cleaned up
|
||||
Path stagingBasePath =
|
||||
new Path(FSUtils.getRootDir(util.getConfiguration()), HConstants.BULKLOAD_STAGING_DIR_NAME);
|
||||
new Path(FSUtils.getRootDir(util.getConfiguration()), HConstants.BULKLOAD_STAGING_DIR_NAME);
|
||||
FileSystem fs = util.getTestFileSystem();
|
||||
if (fs.exists(stagingBasePath)) {
|
||||
FileStatus[] files = fs.listStatus(stagingBasePath);
|
||||
|
@ -419,7 +413,7 @@ public class TestLoadIncrementalHFiles {
|
|||
Path familyDir = new Path(dir, Bytes.toString(FAMILY));
|
||||
// table has these split points
|
||||
byte[][] tableSplitKeys = new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("fff"),
|
||||
Bytes.toBytes("jjj"), Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), };
|
||||
Bytes.toBytes("jjj"), Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"), };
|
||||
|
||||
// creating an hfile that has values that span the split points.
|
||||
byte[] from = Bytes.toBytes("ddd");
|
||||
|
@ -432,13 +426,11 @@ public class TestLoadIncrementalHFiles {
|
|||
TableDescriptor htd = buildHTD(tableName, BloomType.NONE);
|
||||
util.getAdmin().createTable(htd, tableSplitKeys);
|
||||
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
|
||||
String[] args = { dir.toString(), tableName.toString() };
|
||||
loader.run(args);
|
||||
BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(tableName, dir);
|
||||
|
||||
Table table = util.getConnection().getTable(tableName);
|
||||
try {
|
||||
assertEquals(expectedRows, util.countRows(table));
|
||||
assertEquals(expectedRows, countRows(table));
|
||||
HFileTestUtil.verifyTags(table);
|
||||
} finally {
|
||||
table.close();
|
||||
|
@ -454,16 +446,16 @@ public class TestLoadIncrementalHFiles {
|
|||
public void testNonexistentColumnFamilyLoad() throws Exception {
|
||||
String testName = tn.getMethodName();
|
||||
byte[][][] hFileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("ccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, };
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaa"), Bytes.toBytes("ccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("ooo") }, };
|
||||
|
||||
byte[] TABLE = Bytes.toBytes("mytable_" + testName);
|
||||
// set real family name to upper case in purpose to simulate the case that
|
||||
// family name in HFiles is invalid
|
||||
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE))
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.of(Bytes.toBytes(new String(FAMILY).toUpperCase(Locale.ROOT))))
|
||||
.build();
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder
|
||||
.of(Bytes.toBytes(new String(FAMILY).toUpperCase(Locale.ROOT))))
|
||||
.build();
|
||||
|
||||
try {
|
||||
runTest(testName, htd, true, SPLIT_KEYS, hFileRanges, false, false, 2);
|
||||
|
@ -474,7 +466,7 @@ public class TestLoadIncrementalHFiles {
|
|||
String errMsg = e.getMessage();
|
||||
assertTrue(
|
||||
"Incorrect exception message, expected message: [" + EXPECTED_MSG_FOR_NON_EXISTING_FAMILY +
|
||||
"], current message: [" + errMsg + "]",
|
||||
"], current message: [" + errMsg + "]",
|
||||
errMsg.contains(EXPECTED_MSG_FOR_NON_EXISTING_FAMILY));
|
||||
}
|
||||
}
|
||||
|
@ -517,10 +509,8 @@ public class TestLoadIncrementalHFiles {
|
|||
} else {
|
||||
table = util.getConnection().getTable(TableName.valueOf(tableName));
|
||||
}
|
||||
|
||||
final String[] args = { dir.toString(), tableName };
|
||||
new LoadIncrementalHFiles(util.getConfiguration()).run(args);
|
||||
assertEquals(500, util.countRows(table));
|
||||
BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(TableName.valueOf(tableName), dir);
|
||||
assertEquals(500, countRows(table));
|
||||
} finally {
|
||||
if (table != null) {
|
||||
table.close();
|
||||
|
@ -560,7 +550,7 @@ public class TestLoadIncrementalHFiles {
|
|||
Path bottomOut = new Path(dir, "bottom.out");
|
||||
Path topOut = new Path(dir, "top.out");
|
||||
|
||||
LoadIncrementalHFiles.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
|
||||
BulkLoadHFilesTool.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
|
||||
Bytes.toBytes("ggg"), bottomOut, topOut);
|
||||
|
||||
int rowCount = verifyHFile(bottomOut);
|
||||
|
@ -594,14 +584,14 @@ public class TestLoadIncrementalHFiles {
|
|||
FileSystem fs = util.getTestFileSystem();
|
||||
Path testIn = new Path(dir, "testhfile");
|
||||
ColumnFamilyDescriptor familyDesc =
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setDataBlockEncoding(cfEncoding).build();
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setDataBlockEncoding(cfEncoding).build();
|
||||
HFileTestUtil.createHFileWithDataBlockEncoding(util.getConfiguration(), fs, testIn,
|
||||
bulkloadEncoding, FAMILY, QUALIFIER, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), 1000);
|
||||
|
||||
Path bottomOut = new Path(dir, "bottom.out");
|
||||
Path topOut = new Path(dir, "top.out");
|
||||
|
||||
LoadIncrementalHFiles.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
|
||||
BulkLoadHFilesTool.splitStoreFile(util.getConfiguration(), testIn, familyDesc,
|
||||
Bytes.toBytes("ggg"), bottomOut, topOut);
|
||||
|
||||
int rowCount = verifyHFile(bottomOut);
|
||||
|
@ -612,7 +602,7 @@ public class TestLoadIncrementalHFiles {
|
|||
private int verifyHFile(Path p) throws IOException {
|
||||
Configuration conf = util.getConfiguration();
|
||||
HFile.Reader reader =
|
||||
HFile.createReader(p.getFileSystem(conf), p, new CacheConfig(conf), true, conf);
|
||||
HFile.createReader(p.getFileSystem(conf), p, new CacheConfig(conf), true, conf);
|
||||
reader.loadFileInfo();
|
||||
HFileScanner scanner = reader.getScanner(false, false);
|
||||
scanner.seekTo();
|
||||
|
@ -682,7 +672,7 @@ public class TestLoadIncrementalHFiles {
|
|||
last = "w";
|
||||
addStartEndKeysForTest(map, Bytes.toBytes(first), Bytes.toBytes(last));
|
||||
|
||||
byte[][] keysArray = LoadIncrementalHFiles.inferBoundaries(map);
|
||||
byte[][] keysArray = BulkLoadHFilesTool.inferBoundaries(map);
|
||||
byte[][] compare = new byte[3][];
|
||||
compare[0] = Bytes.toBytes("m");
|
||||
compare[1] = Bytes.toBytes("r");
|
||||
|
@ -709,22 +699,21 @@ public class TestLoadIncrementalHFiles {
|
|||
FAMILY, QUALIFIER, from, to, 1000);
|
||||
}
|
||||
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
|
||||
String[] args = { dir.toString(), "mytable_testLoadTooMayHFiles" };
|
||||
try {
|
||||
loader.run(args);
|
||||
BulkLoadHFiles.create(util.getConfiguration())
|
||||
.bulkLoad(TableName.valueOf("mytable_testLoadTooMayHFiles"), dir);
|
||||
fail("Bulk loading too many files should fail");
|
||||
} catch (IOException ie) {
|
||||
assertTrue(ie.getMessage()
|
||||
.contains("Trying to load more than " + MAX_FILES_PER_REGION_PER_FAMILY + " hfiles"));
|
||||
.contains("Trying to load more than " + MAX_FILES_PER_REGION_PER_FAMILY + " hfiles"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test(expected = TableNotFoundException.class)
|
||||
public void testWithoutAnExistingTableAndCreateTableSetToNo() throws Exception {
|
||||
Configuration conf = util.getConfiguration();
|
||||
conf.set(LoadIncrementalHFiles.CREATE_TABLE_CONF_KEY, "no");
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
|
||||
conf.set(BulkLoadHFiles.CREATE_TABLE_CONF_KEY, "no");
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf);
|
||||
String[] args = { "directory", "nonExistingTable" };
|
||||
loader.run(args);
|
||||
}
|
||||
|
@ -741,19 +730,11 @@ public class TestLoadIncrementalHFiles {
|
|||
byte[] to = Bytes.toBytes("end");
|
||||
Configuration conf = util.getConfiguration();
|
||||
String tableName = tn.getMethodName();
|
||||
Table table = util.createTable(TableName.valueOf(tableName), family);
|
||||
HFileTestUtil.createHFile(conf, fs, new Path(familyDir, "hfile"), Bytes.toBytes(family),
|
||||
QUALIFIER, from, to, 1000);
|
||||
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
|
||||
String[] args = { dir.toString(), tableName };
|
||||
try {
|
||||
loader.run(args);
|
||||
assertEquals(1000, util.countRows(table));
|
||||
} finally {
|
||||
if (null != table) {
|
||||
table.close();
|
||||
}
|
||||
try (Table table = util.createTable(TableName.valueOf(tableName), family)) {
|
||||
HFileTestUtil.createHFile(conf, fs, new Path(familyDir, "hfile"), Bytes.toBytes(family),
|
||||
QUALIFIER, from, to, 1000);
|
||||
BulkLoadHFiles.create(conf).bulkLoad(table.getName(), dir);
|
||||
assertEquals(1000, countRows(table));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,486 @@
|
|||
/**
|
||||
* 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.tool;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyBoolean;
|
||||
import static org.mockito.ArgumentMatchers.anyList;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.spy;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Deque;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.AsyncClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
||||
/**
|
||||
* Test cases for the atomic load error handling of the bulk load functionality.
|
||||
*/
|
||||
@Category({ MiscTests.class, LargeTests.class })
|
||||
public class TestBulkLoadHFilesSplitRecovery {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestBulkLoadHFilesSplitRecovery.class);
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestHRegionServerBulkLoad.class);
|
||||
|
||||
static HBaseTestingUtility util;
|
||||
// used by secure subclass
|
||||
static boolean useSecure = false;
|
||||
|
||||
final static int NUM_CFS = 10;
|
||||
final static byte[] QUAL = Bytes.toBytes("qual");
|
||||
final static int ROWCOUNT = 100;
|
||||
|
||||
private final static byte[][] families = new byte[NUM_CFS][];
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
static {
|
||||
for (int i = 0; i < NUM_CFS; i++) {
|
||||
families[i] = Bytes.toBytes(family(i));
|
||||
}
|
||||
}
|
||||
|
||||
static byte[] rowkey(int i) {
|
||||
return Bytes.toBytes(String.format("row_%08d", i));
|
||||
}
|
||||
|
||||
static String family(int i) {
|
||||
return String.format("family_%04d", i);
|
||||
}
|
||||
|
||||
static byte[] value(int i) {
|
||||
return Bytes.toBytes(String.format("%010d", i));
|
||||
}
|
||||
|
||||
public static void buildHFiles(FileSystem fs, Path dir, int value) throws IOException {
|
||||
byte[] val = value(value);
|
||||
for (int i = 0; i < NUM_CFS; i++) {
|
||||
Path testIn = new Path(dir, family(i));
|
||||
|
||||
TestHRegionServerBulkLoad.createHFile(fs, new Path(testIn, "hfile_" + i),
|
||||
Bytes.toBytes(family(i)), QUAL, val, ROWCOUNT);
|
||||
}
|
||||
}
|
||||
|
||||
private TableDescriptor createTableDesc(TableName name, int cfs) {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
|
||||
IntStream.range(0, cfs).mapToObj(i -> ColumnFamilyDescriptorBuilder.of(family(i)))
|
||||
.forEachOrdered(builder::setColumnFamily);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a table with given table name and specified number of column families if the table does
|
||||
* not already exist.
|
||||
*/
|
||||
private void setupTable(final Connection connection, TableName table, int cfs)
|
||||
throws IOException {
|
||||
try {
|
||||
LOG.info("Creating table " + table);
|
||||
try (Admin admin = connection.getAdmin()) {
|
||||
admin.createTable(createTableDesc(table, cfs));
|
||||
}
|
||||
} catch (TableExistsException tee) {
|
||||
LOG.info("Table " + table + " already exists");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a table with given table name,specified number of column families<br>
|
||||
* and splitkeys if the table does not already exist.
|
||||
* @param table
|
||||
* @param cfs
|
||||
* @param SPLIT_KEYS
|
||||
*/
|
||||
private void setupTableWithSplitkeys(TableName table, int cfs, byte[][] SPLIT_KEYS)
|
||||
throws IOException {
|
||||
try {
|
||||
LOG.info("Creating table " + table);
|
||||
util.createTable(createTableDesc(table, cfs), SPLIT_KEYS);
|
||||
} catch (TableExistsException tee) {
|
||||
LOG.info("Table " + table + " already exists");
|
||||
}
|
||||
}
|
||||
|
||||
private Path buildBulkFiles(TableName table, int value) throws Exception {
|
||||
Path dir = util.getDataTestDirOnTestFS(table.getNameAsString());
|
||||
Path bulk1 = new Path(dir, table.getNameAsString() + value);
|
||||
FileSystem fs = util.getTestFileSystem();
|
||||
buildHFiles(fs, bulk1, value);
|
||||
return bulk1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Populate table with known values.
|
||||
*/
|
||||
private void populateTable(final Connection connection, TableName table, int value)
|
||||
throws Exception {
|
||||
// create HFiles for different column families
|
||||
Path dir = buildBulkFiles(table, value);
|
||||
BulkLoadHFiles.create(util.getConfiguration()).bulkLoad(table, dir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Split the known table in half. (this is hard coded for this test suite)
|
||||
*/
|
||||
private void forceSplit(TableName table) {
|
||||
try {
|
||||
// need to call regions server to by synchronous but isn't visible.
|
||||
HRegionServer hrs = util.getRSForFirstRegionInTable(table);
|
||||
|
||||
for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
|
||||
if (hri.getTable().equals(table)) {
|
||||
util.getAdmin().splitRegionAsync(hri.getRegionName(), rowkey(ROWCOUNT / 2));
|
||||
// ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2));
|
||||
}
|
||||
}
|
||||
|
||||
// verify that split completed.
|
||||
int regions;
|
||||
do {
|
||||
regions = 0;
|
||||
for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
|
||||
if (hri.getTable().equals(table)) {
|
||||
regions++;
|
||||
}
|
||||
}
|
||||
if (regions != 2) {
|
||||
LOG.info("Taking some time to complete split...");
|
||||
Thread.sleep(250);
|
||||
}
|
||||
} while (regions != 2);
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
util = new HBaseTestingUtility();
|
||||
util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
|
||||
util.startMiniCluster(1);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownCluster() throws Exception {
|
||||
util.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that all columns have the expected value and that there is the expected number of rows.
|
||||
* @throws IOException
|
||||
*/
|
||||
void assertExpectedTable(TableName table, int count, int value) throws IOException {
|
||||
TableDescriptor htd = util.getAdmin().getDescriptor(table);
|
||||
assertNotNull(htd);
|
||||
try (Table t = util.getConnection().getTable(table);
|
||||
ResultScanner sr = t.getScanner(new Scan())) {
|
||||
int i = 0;
|
||||
for (Result r; (r = sr.next()) != null;) {
|
||||
r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
|
||||
.forEach(v -> assertArrayEquals(value(value), v));
|
||||
i++;
|
||||
}
|
||||
assertEquals(count, i);
|
||||
} catch (IOException e) {
|
||||
fail("Failed due to exception");
|
||||
}
|
||||
}
|
||||
|
||||
private static <T> CompletableFuture<T> failedFuture(Throwable error) {
|
||||
CompletableFuture<T> future = new CompletableFuture<>();
|
||||
future.completeExceptionally(error);
|
||||
return future;
|
||||
}
|
||||
|
||||
private static AsyncClusterConnection mockAndInjectError(AsyncClusterConnection conn) {
|
||||
AsyncClusterConnection errConn = spy(conn);
|
||||
doReturn(failedFuture(new IOException("injecting bulk load error"))).when(errConn)
|
||||
.bulkLoad(any(), anyList(), any(), anyBoolean(), any(), any(), anyBoolean());
|
||||
return errConn;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that shows that exception thrown from the RS side will result in an exception on the
|
||||
* LIHFile client.
|
||||
*/
|
||||
@Test(expected = IOException.class)
|
||||
public void testBulkLoadPhaseFailure() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
final AtomicInteger attemptedCalls = new AtomicInteger();
|
||||
Configuration conf = new Configuration(util.getConfiguration());
|
||||
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf) {
|
||||
|
||||
@Override
|
||||
protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
|
||||
Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
|
||||
boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
|
||||
AsyncClusterConnection c =
|
||||
attemptedCalls.incrementAndGet() == 1 ? mockAndInjectError(conn) : conn;
|
||||
super.bulkLoadPhase(c, tableName, queue, regionGroups, copyFiles, item2RegionMap);
|
||||
}
|
||||
};
|
||||
Path dir = buildBulkFiles(table, 1);
|
||||
loader.bulkLoad(table, dir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that shows that exception thrown from the RS side will result in the expected number of
|
||||
* retries set by ${@link HConstants#HBASE_CLIENT_RETRIES_NUMBER} when
|
||||
* ${@link BulkLoadHFiles#RETRY_ON_IO_EXCEPTION} is set
|
||||
*/
|
||||
@Test
|
||||
public void testRetryOnIOException() throws Exception {
|
||||
TableName table = TableName.valueOf(name.getMethodName());
|
||||
AtomicInteger calls = new AtomicInteger(0);
|
||||
setupTable(util.getConnection(), table, 10);
|
||||
Configuration conf = new Configuration(util.getConfiguration());
|
||||
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
|
||||
conf.setBoolean(BulkLoadHFiles.RETRY_ON_IO_EXCEPTION, true);
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(conf) {
|
||||
|
||||
@Override
|
||||
protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
|
||||
Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
|
||||
boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
|
||||
if (calls.get() < conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)) {
|
||||
calls.incrementAndGet();
|
||||
super.bulkLoadPhase(mockAndInjectError(conn), tableName, queue, regionGroups, copyFiles,
|
||||
item2RegionMap);
|
||||
} else {
|
||||
super.bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, item2RegionMap);
|
||||
}
|
||||
}
|
||||
};
|
||||
Path dir = buildBulkFiles(table, 1);
|
||||
loader.bulkLoad(table, dir);
|
||||
assertEquals(calls.get(), 2);
|
||||
}
|
||||
|
||||
/**
|
||||
* This test exercises the path where there is a split after initial validation but before the
|
||||
* atomic bulk load call. We cannot use presplitting to test this path, so we actually inject a
|
||||
* split just before the atomic region load.
|
||||
*/
|
||||
@Test
|
||||
public void testSplitWhileBulkLoadPhase() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
setupTable(util.getConnection(), table, 10);
|
||||
populateTable(util.getConnection(), table, 1);
|
||||
assertExpectedTable(table, ROWCOUNT, 1);
|
||||
|
||||
// Now let's cause trouble. This will occur after checks and cause bulk
|
||||
// files to fail when attempt to atomically import. This is recoverable.
|
||||
final AtomicInteger attemptedCalls = new AtomicInteger();
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(util.getConfiguration()) {
|
||||
|
||||
@Override
|
||||
protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
|
||||
Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
|
||||
boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
|
||||
int i = attemptedCalls.incrementAndGet();
|
||||
if (i == 1) {
|
||||
// On first attempt force a split.
|
||||
forceSplit(table);
|
||||
}
|
||||
super.bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, item2RegionMap);
|
||||
}
|
||||
};
|
||||
|
||||
// create HFiles for different column families
|
||||
Path dir = buildBulkFiles(table, 2);
|
||||
loader.bulkLoad(table, dir);
|
||||
|
||||
// check that data was loaded
|
||||
// The three expected attempts are 1) failure because need to split, 2)
|
||||
// load of split top 3) load of split bottom
|
||||
assertEquals(3, attemptedCalls.get());
|
||||
assertExpectedTable(table, ROWCOUNT, 2);
|
||||
}
|
||||
|
||||
/**
|
||||
* This test splits a table and attempts to bulk load. The bulk import files should be split
|
||||
* before atomically importing.
|
||||
*/
|
||||
@Test
|
||||
public void testGroupOrSplitPresplit() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
setupTable(util.getConnection(), table, 10);
|
||||
populateTable(util.getConnection(), table, 1);
|
||||
assertExpectedTable(util.getConnection(), table, ROWCOUNT, 1);
|
||||
forceSplit(table);
|
||||
|
||||
final AtomicInteger countedLqis = new AtomicInteger();
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(util.getConfiguration()) {
|
||||
|
||||
@Override
|
||||
protected Pair<List<LoadQueueItem>, String> groupOrSplit(AsyncClusterConnection conn,
|
||||
TableName tableName, Multimap<ByteBuffer, LoadQueueItem> regionGroups, LoadQueueItem item,
|
||||
List<Pair<byte[], byte[]>> startEndKeys) throws IOException {
|
||||
Pair<List<LoadQueueItem>, String> lqis =
|
||||
super.groupOrSplit(conn, tableName, regionGroups, item, startEndKeys);
|
||||
if (lqis != null && lqis.getFirst() != null) {
|
||||
countedLqis.addAndGet(lqis.getFirst().size());
|
||||
}
|
||||
return lqis;
|
||||
}
|
||||
};
|
||||
|
||||
// create HFiles for different column families
|
||||
Path dir = buildBulkFiles(table, 2);
|
||||
loader.bulkLoad(table, dir);
|
||||
assertExpectedTable(util.getConnection(), table, ROWCOUNT, 2);
|
||||
assertEquals(20, countedLqis.get());
|
||||
}
|
||||
|
||||
/**
|
||||
* This test creates a table with many small regions. The bulk load files would be splitted
|
||||
* multiple times before all of them can be loaded successfully.
|
||||
*/
|
||||
@Test
|
||||
public void testSplitTmpFileCleanUp() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000010"),
|
||||
Bytes.toBytes("row_00000020"), Bytes.toBytes("row_00000030"), Bytes.toBytes("row_00000040"),
|
||||
Bytes.toBytes("row_00000050") };
|
||||
setupTableWithSplitkeys(table, 10, SPLIT_KEYS);
|
||||
|
||||
BulkLoadHFiles loader = BulkLoadHFiles.create(util.getConfiguration());
|
||||
|
||||
// create HFiles
|
||||
Path dir = buildBulkFiles(table, 2);
|
||||
loader.bulkLoad(table, dir);
|
||||
// family path
|
||||
Path tmpPath = new Path(dir, family(0));
|
||||
// TMP_DIR under family path
|
||||
tmpPath = new Path(tmpPath, BulkLoadHFilesTool.TMP_DIR);
|
||||
FileSystem fs = dir.getFileSystem(util.getConfiguration());
|
||||
// HFiles have been splitted, there is TMP_DIR
|
||||
assertTrue(fs.exists(tmpPath));
|
||||
// TMP_DIR should have been cleaned-up
|
||||
assertNull(BulkLoadHFilesTool.TMP_DIR + " should be empty.", FSUtils.listStatus(fs, tmpPath));
|
||||
assertExpectedTable(util.getConnection(), table, ROWCOUNT, 2);
|
||||
}
|
||||
|
||||
/**
|
||||
* This simulates an remote exception which should cause LIHF to exit with an exception.
|
||||
*/
|
||||
@Test(expected = IOException.class)
|
||||
public void testGroupOrSplitFailure() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
setupTable(util.getConnection(), tableName, 10);
|
||||
BulkLoadHFilesTool loader = new BulkLoadHFilesTool(util.getConfiguration()) {
|
||||
|
||||
private int i = 0;
|
||||
|
||||
@Override
|
||||
protected Pair<List<LoadQueueItem>, String> groupOrSplit(AsyncClusterConnection conn,
|
||||
TableName tableName, Multimap<ByteBuffer, LoadQueueItem> regionGroups, LoadQueueItem item,
|
||||
List<Pair<byte[], byte[]>> startEndKeys) throws IOException {
|
||||
i++;
|
||||
|
||||
if (i == 5) {
|
||||
throw new IOException("failure");
|
||||
}
|
||||
return super.groupOrSplit(conn, tableName, regionGroups, item, startEndKeys);
|
||||
}
|
||||
};
|
||||
|
||||
// create HFiles for different column families
|
||||
Path dir = buildBulkFiles(tableName, 1);
|
||||
loader.bulkLoad(tableName, dir);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that all columns have the expected value and that there is the expected number of rows.
|
||||
*/
|
||||
void assertExpectedTable(final Connection connection, TableName table, int count, int value)
|
||||
throws IOException {
|
||||
TableDescriptor htd = util.getAdmin().getDescriptor(table);
|
||||
assertNotNull(htd);
|
||||
try (Table t = connection.getTable(table); ResultScanner sr = t.getScanner(new Scan())) {
|
||||
int i = 0;
|
||||
for (Result r; (r = sr.next()) != null;) {
|
||||
r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
|
||||
.forEach(v -> assertArrayEquals(value(value), v));
|
||||
i++;
|
||||
}
|
||||
assertEquals(count, i);
|
||||
} catch (IOException e) {
|
||||
fail("Failed due to exception");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,630 +0,0 @@
|
|||
/**
|
||||
* 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.tool;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collection;
|
||||
import java.util.Deque;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClientServiceCallable;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.log.HBaseMarkers;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.mockito.Mockito;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
|
||||
|
||||
/**
|
||||
* Test cases for the atomic load error handling of the bulk load functionality.
|
||||
*/
|
||||
@Category({ MiscTests.class, LargeTests.class })
|
||||
public class TestLoadIncrementalHFilesSplitRecovery {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestLoadIncrementalHFilesSplitRecovery.class);
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestHRegionServerBulkLoad.class);
|
||||
|
||||
static HBaseTestingUtility util;
|
||||
// used by secure subclass
|
||||
static boolean useSecure = false;
|
||||
|
||||
final static int NUM_CFS = 10;
|
||||
final static byte[] QUAL = Bytes.toBytes("qual");
|
||||
final static int ROWCOUNT = 100;
|
||||
|
||||
private final static byte[][] families = new byte[NUM_CFS][];
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
static {
|
||||
for (int i = 0; i < NUM_CFS; i++) {
|
||||
families[i] = Bytes.toBytes(family(i));
|
||||
}
|
||||
}
|
||||
|
||||
static byte[] rowkey(int i) {
|
||||
return Bytes.toBytes(String.format("row_%08d", i));
|
||||
}
|
||||
|
||||
static String family(int i) {
|
||||
return String.format("family_%04d", i);
|
||||
}
|
||||
|
||||
static byte[] value(int i) {
|
||||
return Bytes.toBytes(String.format("%010d", i));
|
||||
}
|
||||
|
||||
public static void buildHFiles(FileSystem fs, Path dir, int value) throws IOException {
|
||||
byte[] val = value(value);
|
||||
for (int i = 0; i < NUM_CFS; i++) {
|
||||
Path testIn = new Path(dir, family(i));
|
||||
|
||||
TestHRegionServerBulkLoad.createHFile(fs, new Path(testIn, "hfile_" + i),
|
||||
Bytes.toBytes(family(i)), QUAL, val, ROWCOUNT);
|
||||
}
|
||||
}
|
||||
|
||||
private TableDescriptor createTableDesc(TableName name, int cfs) {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
|
||||
IntStream.range(0, cfs).mapToObj(i -> ColumnFamilyDescriptorBuilder.of(family(i)))
|
||||
.forEachOrdered(builder::setColumnFamily);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a table with given table name and specified number of column families if the table does
|
||||
* not already exist.
|
||||
*/
|
||||
private void setupTable(final Connection connection, TableName table, int cfs)
|
||||
throws IOException {
|
||||
try {
|
||||
LOG.info("Creating table " + table);
|
||||
try (Admin admin = connection.getAdmin()) {
|
||||
admin.createTable(createTableDesc(table, cfs));
|
||||
}
|
||||
} catch (TableExistsException tee) {
|
||||
LOG.info("Table " + table + " already exists");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a table with given table name,specified number of column families<br>
|
||||
* and splitkeys if the table does not already exist.
|
||||
* @param table
|
||||
* @param cfs
|
||||
* @param SPLIT_KEYS
|
||||
*/
|
||||
private void setupTableWithSplitkeys(TableName table, int cfs, byte[][] SPLIT_KEYS)
|
||||
throws IOException {
|
||||
try {
|
||||
LOG.info("Creating table " + table);
|
||||
util.createTable(createTableDesc(table, cfs), SPLIT_KEYS);
|
||||
} catch (TableExistsException tee) {
|
||||
LOG.info("Table " + table + " already exists");
|
||||
}
|
||||
}
|
||||
|
||||
private Path buildBulkFiles(TableName table, int value) throws Exception {
|
||||
Path dir = util.getDataTestDirOnTestFS(table.getNameAsString());
|
||||
Path bulk1 = new Path(dir, table.getNameAsString() + value);
|
||||
FileSystem fs = util.getTestFileSystem();
|
||||
buildHFiles(fs, bulk1, value);
|
||||
return bulk1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Populate table with known values.
|
||||
*/
|
||||
private void populateTable(final Connection connection, TableName table, int value)
|
||||
throws Exception {
|
||||
// create HFiles for different column families
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
|
||||
Path bulk1 = buildBulkFiles(table, value);
|
||||
try (Table t = connection.getTable(table);
|
||||
RegionLocator locator = connection.getRegionLocator(table);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
lih.doBulkLoad(bulk1, admin, t, locator);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Split the known table in half. (this is hard coded for this test suite)
|
||||
*/
|
||||
private void forceSplit(TableName table) {
|
||||
try {
|
||||
// need to call regions server to by synchronous but isn't visible.
|
||||
HRegionServer hrs = util.getRSForFirstRegionInTable(table);
|
||||
|
||||
for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
|
||||
if (hri.getTable().equals(table)) {
|
||||
util.getAdmin().splitRegionAsync(hri.getRegionName(), rowkey(ROWCOUNT / 2));
|
||||
// ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2));
|
||||
}
|
||||
}
|
||||
|
||||
// verify that split completed.
|
||||
int regions;
|
||||
do {
|
||||
regions = 0;
|
||||
for (RegionInfo hri : ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
|
||||
if (hri.getTable().equals(table)) {
|
||||
regions++;
|
||||
}
|
||||
}
|
||||
if (regions != 2) {
|
||||
LOG.info("Taking some time to complete split...");
|
||||
Thread.sleep(250);
|
||||
}
|
||||
} while (regions != 2);
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
util = new HBaseTestingUtility();
|
||||
util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
|
||||
util.startMiniCluster(1);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownCluster() throws Exception {
|
||||
util.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that all columns have the expected value and that there is the expected number of rows.
|
||||
* @throws IOException
|
||||
*/
|
||||
void assertExpectedTable(TableName table, int count, int value) throws IOException {
|
||||
TableDescriptor htd = util.getAdmin().getDescriptor(table);
|
||||
assertNotNull(htd);
|
||||
try (Table t = util.getConnection().getTable(table);
|
||||
ResultScanner sr = t.getScanner(new Scan())) {
|
||||
int i = 0;
|
||||
for (Result r; (r = sr.next()) != null;) {
|
||||
r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
|
||||
.forEach(v -> assertArrayEquals(value(value), v));
|
||||
i++;
|
||||
}
|
||||
assertEquals(count, i);
|
||||
} catch (IOException e) {
|
||||
fail("Failed due to exception");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that shows that exception thrown from the RS side will result in an exception on the
|
||||
* LIHFile client.
|
||||
*/
|
||||
@Test(expected = IOException.class)
|
||||
public void testBulkLoadPhaseFailure() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
final AtomicInteger attmptedCalls = new AtomicInteger();
|
||||
final AtomicInteger failedCalls = new AtomicInteger();
|
||||
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
|
||||
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
|
||||
setupTable(connection, table, 10);
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
|
||||
@Override
|
||||
protected List<LoadQueueItem> tryAtomicRegionLoad(
|
||||
ClientServiceCallable<byte[]> serviceCallable, TableName tableName, final byte[] first,
|
||||
Collection<LoadQueueItem> lqis) throws IOException {
|
||||
int i = attmptedCalls.incrementAndGet();
|
||||
if (i == 1) {
|
||||
Connection errConn;
|
||||
try {
|
||||
errConn = getMockedConnection(util.getConfiguration());
|
||||
serviceCallable = this.buildClientServiceCallable(errConn, table, first, lqis, true);
|
||||
} catch (Exception e) {
|
||||
LOG.error(HBaseMarkers.FATAL, "mocking cruft, should never happen", e);
|
||||
throw new RuntimeException("mocking cruft, should never happen");
|
||||
}
|
||||
failedCalls.incrementAndGet();
|
||||
return super.tryAtomicRegionLoad(serviceCallable, tableName, first, lqis);
|
||||
}
|
||||
|
||||
return super.tryAtomicRegionLoad(serviceCallable, tableName, first, lqis);
|
||||
}
|
||||
};
|
||||
try {
|
||||
// create HFiles for different column families
|
||||
Path dir = buildBulkFiles(table, 1);
|
||||
try (Table t = connection.getTable(table);
|
||||
RegionLocator locator = connection.getRegionLocator(table);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
lih.doBulkLoad(dir, admin, t, locator);
|
||||
}
|
||||
} finally {
|
||||
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
|
||||
}
|
||||
fail("doBulkLoad should have thrown an exception");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that shows that exception thrown from the RS side will result in the expected number of
|
||||
* retries set by ${@link HConstants#HBASE_CLIENT_RETRIES_NUMBER} when
|
||||
* ${@link LoadIncrementalHFiles#RETRY_ON_IO_EXCEPTION} is set
|
||||
*/
|
||||
@Test
|
||||
public void testRetryOnIOException() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
final AtomicInteger calls = new AtomicInteger(0);
|
||||
final Connection conn = ConnectionFactory.createConnection(util.getConfiguration());
|
||||
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
|
||||
util.getConfiguration().setBoolean(LoadIncrementalHFiles.RETRY_ON_IO_EXCEPTION, true);
|
||||
final LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
|
||||
@Override
|
||||
protected List<LoadQueueItem> tryAtomicRegionLoad(
|
||||
ClientServiceCallable<byte[]> serverCallable, TableName tableName, final byte[] first,
|
||||
Collection<LoadQueueItem> lqis) throws IOException {
|
||||
if (calls.get() < util.getConfiguration().getInt(
|
||||
HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)) {
|
||||
ClientServiceCallable<byte[]> newServerCallable = new ClientServiceCallable<byte[]>(conn,
|
||||
tableName, first, new RpcControllerFactory(util.getConfiguration()).newController(),
|
||||
HConstants.PRIORITY_UNSET) {
|
||||
@Override
|
||||
public byte[] rpcCall() throws Exception {
|
||||
throw new IOException("Error calling something on RegionServer");
|
||||
}
|
||||
};
|
||||
calls.getAndIncrement();
|
||||
return super.tryAtomicRegionLoad(newServerCallable, tableName, first, lqis);
|
||||
} else {
|
||||
return super.tryAtomicRegionLoad(serverCallable, tableName, first, lqis);
|
||||
}
|
||||
}
|
||||
};
|
||||
setupTable(conn, table, 10);
|
||||
Path dir = buildBulkFiles(table, 1);
|
||||
lih.doBulkLoad(dir, conn.getAdmin(), conn.getTable(table), conn.getRegionLocator(table));
|
||||
assertEquals(calls.get(), 2);
|
||||
util.getConfiguration().setBoolean(LoadIncrementalHFiles.RETRY_ON_IO_EXCEPTION, false);
|
||||
}
|
||||
|
||||
private ClusterConnection getMockedConnection(final Configuration conf)
|
||||
throws IOException, org.apache.hbase.thirdparty.com.google.protobuf.ServiceException {
|
||||
ServerName sn = ServerName.valueOf("example.org", 1234, 0);
|
||||
RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
|
||||
ClientProtos.ClientService.BlockingInterface client =
|
||||
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
|
||||
Mockito
|
||||
.when(
|
||||
client.bulkLoadHFile((RpcController) Mockito.any(), (BulkLoadHFileRequest) Mockito.any()))
|
||||
.thenThrow(new ServiceException(new IOException("injecting bulk load error")));
|
||||
return HConnectionTestingUtility.getMockedConnectionAndDecorate(conf, null, client, sn, hri);
|
||||
}
|
||||
|
||||
/**
|
||||
* This test exercises the path where there is a split after initial validation but before the
|
||||
* atomic bulk load call. We cannot use presplitting to test this path, so we actually inject a
|
||||
* split just before the atomic region load.
|
||||
*/
|
||||
@Test
|
||||
public void testSplitWhileBulkLoadPhase() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
|
||||
setupTable(connection, table, 10);
|
||||
populateTable(connection, table, 1);
|
||||
assertExpectedTable(table, ROWCOUNT, 1);
|
||||
|
||||
// Now let's cause trouble. This will occur after checks and cause bulk
|
||||
// files to fail when attempt to atomically import. This is recoverable.
|
||||
final AtomicInteger attemptedCalls = new AtomicInteger();
|
||||
LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles(util.getConfiguration()) {
|
||||
@Override
|
||||
protected void bulkLoadPhase(final Table htable, final Connection conn,
|
||||
ExecutorService pool, Deque<LoadQueueItem> queue,
|
||||
final Multimap<ByteBuffer, LoadQueueItem> regionGroups, boolean copyFile,
|
||||
Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
|
||||
int i = attemptedCalls.incrementAndGet();
|
||||
if (i == 1) {
|
||||
// On first attempt force a split.
|
||||
forceSplit(table);
|
||||
}
|
||||
super.bulkLoadPhase(htable, conn, pool, queue, regionGroups, copyFile, item2RegionMap);
|
||||
}
|
||||
};
|
||||
|
||||
// create HFiles for different column families
|
||||
try (Table t = connection.getTable(table);
|
||||
RegionLocator locator = connection.getRegionLocator(table);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
Path bulk = buildBulkFiles(table, 2);
|
||||
lih2.doBulkLoad(bulk, admin, t, locator);
|
||||
}
|
||||
|
||||
// check that data was loaded
|
||||
// The three expected attempts are 1) failure because need to split, 2)
|
||||
// load of split top 3) load of split bottom
|
||||
assertEquals(3, attemptedCalls.get());
|
||||
assertExpectedTable(table, ROWCOUNT, 2);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This test splits a table and attempts to bulk load. The bulk import files should be split
|
||||
* before atomically importing.
|
||||
*/
|
||||
@Test
|
||||
public void testGroupOrSplitPresplit() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
|
||||
setupTable(connection, table, 10);
|
||||
populateTable(connection, table, 1);
|
||||
assertExpectedTable(connection, table, ROWCOUNT, 1);
|
||||
forceSplit(table);
|
||||
|
||||
final AtomicInteger countedLqis = new AtomicInteger();
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
|
||||
@Override
|
||||
protected Pair<List<LoadQueueItem>, String> groupOrSplit(
|
||||
Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item,
|
||||
final Table htable, final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
|
||||
Pair<List<LoadQueueItem>, String> lqis =
|
||||
super.groupOrSplit(regionGroups, item, htable, startEndKeys);
|
||||
if (lqis != null && lqis.getFirst() != null) {
|
||||
countedLqis.addAndGet(lqis.getFirst().size());
|
||||
}
|
||||
return lqis;
|
||||
}
|
||||
};
|
||||
|
||||
// create HFiles for different column families
|
||||
Path bulk = buildBulkFiles(table, 2);
|
||||
try (Table t = connection.getTable(table);
|
||||
RegionLocator locator = connection.getRegionLocator(table);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
lih.doBulkLoad(bulk, admin, t, locator);
|
||||
}
|
||||
assertExpectedTable(connection, table, ROWCOUNT, 2);
|
||||
assertEquals(20, countedLqis.get());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This test creates a table with many small regions. The bulk load files would be splitted
|
||||
* multiple times before all of them can be loaded successfully.
|
||||
*/
|
||||
@Test
|
||||
public void testSplitTmpFileCleanUp() throws Exception {
|
||||
final TableName table = TableName.valueOf(name.getMethodName());
|
||||
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000010"),
|
||||
Bytes.toBytes("row_00000020"), Bytes.toBytes("row_00000030"), Bytes.toBytes("row_00000040"),
|
||||
Bytes.toBytes("row_00000050") };
|
||||
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
|
||||
setupTableWithSplitkeys(table, 10, SPLIT_KEYS);
|
||||
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
|
||||
|
||||
// create HFiles
|
||||
Path bulk = buildBulkFiles(table, 2);
|
||||
try (Table t = connection.getTable(table);
|
||||
RegionLocator locator = connection.getRegionLocator(table);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
lih.doBulkLoad(bulk, admin, t, locator);
|
||||
}
|
||||
// family path
|
||||
Path tmpPath = new Path(bulk, family(0));
|
||||
// TMP_DIR under family path
|
||||
tmpPath = new Path(tmpPath, LoadIncrementalHFiles.TMP_DIR);
|
||||
FileSystem fs = bulk.getFileSystem(util.getConfiguration());
|
||||
// HFiles have been splitted, there is TMP_DIR
|
||||
assertTrue(fs.exists(tmpPath));
|
||||
// TMP_DIR should have been cleaned-up
|
||||
assertNull(LoadIncrementalHFiles.TMP_DIR + " should be empty.",
|
||||
FSUtils.listStatus(fs, tmpPath));
|
||||
assertExpectedTable(connection, table, ROWCOUNT, 2);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This simulates an remote exception which should cause LIHF to exit with an exception.
|
||||
*/
|
||||
@Test(expected = IOException.class)
|
||||
public void testGroupOrSplitFailure() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
|
||||
setupTable(connection, tableName, 10);
|
||||
|
||||
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
|
||||
int i = 0;
|
||||
|
||||
@Override
|
||||
protected Pair<List<LoadQueueItem>, String> groupOrSplit(
|
||||
Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item,
|
||||
final Table table, final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
|
||||
i++;
|
||||
|
||||
if (i == 5) {
|
||||
throw new IOException("failure");
|
||||
}
|
||||
return super.groupOrSplit(regionGroups, item, table, startEndKeys);
|
||||
}
|
||||
};
|
||||
|
||||
// create HFiles for different column families
|
||||
Path dir = buildBulkFiles(tableName, 1);
|
||||
try (Table t = connection.getTable(tableName);
|
||||
RegionLocator locator = connection.getRegionLocator(tableName);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
lih.doBulkLoad(dir, admin, t, locator);
|
||||
}
|
||||
}
|
||||
|
||||
fail("doBulkLoad should have thrown an exception");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupOrSplitWhenRegionHoleExistsInMeta() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000100") };
|
||||
// Share connection. We were failing to find the table with our new reverse scan because it
|
||||
// looks for first region, not any region -- that is how it works now. The below removes first
|
||||
// region in test. Was reliant on the Connection caching having first region.
|
||||
Connection connection = ConnectionFactory.createConnection(util.getConfiguration());
|
||||
Table table = connection.getTable(tableName);
|
||||
|
||||
setupTableWithSplitkeys(tableName, 10, SPLIT_KEYS);
|
||||
Path dir = buildBulkFiles(tableName, 2);
|
||||
|
||||
final AtomicInteger countedLqis = new AtomicInteger();
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration()) {
|
||||
|
||||
@Override
|
||||
protected Pair<List<LoadQueueItem>, String> groupOrSplit(
|
||||
Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item,
|
||||
final Table htable, final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
|
||||
Pair<List<LoadQueueItem>, String> lqis =
|
||||
super.groupOrSplit(regionGroups, item, htable, startEndKeys);
|
||||
if (lqis != null && lqis.getFirst() != null) {
|
||||
countedLqis.addAndGet(lqis.getFirst().size());
|
||||
}
|
||||
return lqis;
|
||||
}
|
||||
};
|
||||
|
||||
// do bulkload when there is no region hole in hbase:meta.
|
||||
try (Table t = connection.getTable(tableName);
|
||||
RegionLocator locator = connection.getRegionLocator(tableName);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
loader.doBulkLoad(dir, admin, t, locator);
|
||||
} catch (Exception e) {
|
||||
LOG.error("exeception=", e);
|
||||
}
|
||||
// check if all the data are loaded into the table.
|
||||
this.assertExpectedTable(tableName, ROWCOUNT, 2);
|
||||
|
||||
dir = buildBulkFiles(tableName, 3);
|
||||
|
||||
// Mess it up by leaving a hole in the hbase:meta
|
||||
List<RegionInfo> regionInfos = MetaTableAccessor.getTableRegions(connection, tableName);
|
||||
for (RegionInfo regionInfo : regionInfos) {
|
||||
if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
|
||||
MetaTableAccessor.deleteRegion(connection, regionInfo);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
try (Table t = connection.getTable(tableName);
|
||||
RegionLocator locator = connection.getRegionLocator(tableName);
|
||||
Admin admin = connection.getAdmin()) {
|
||||
loader.doBulkLoad(dir, admin, t, locator);
|
||||
} catch (Exception e) {
|
||||
LOG.error("exception=", e);
|
||||
assertTrue("IOException expected", e instanceof IOException);
|
||||
}
|
||||
|
||||
table.close();
|
||||
|
||||
// Make sure at least the one region that still exists can be found.
|
||||
regionInfos = MetaTableAccessor.getTableRegions(connection, tableName);
|
||||
assertTrue(regionInfos.size() >= 1);
|
||||
|
||||
this.assertExpectedTable(connection, tableName, ROWCOUNT, 2);
|
||||
connection.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that all columns have the expected value and that there is the expected number of rows.
|
||||
* @throws IOException
|
||||
*/
|
||||
void assertExpectedTable(final Connection connection, TableName table, int count, int value)
|
||||
throws IOException {
|
||||
TableDescriptor htd = util.getAdmin().getDescriptor(table);
|
||||
assertNotNull(htd);
|
||||
try (Table t = connection.getTable(table); ResultScanner sr = t.getScanner(new Scan())) {
|
||||
int i = 0;
|
||||
for (Result r; (r = sr.next()) != null;) {
|
||||
r.getNoVersionMap().values().stream().flatMap(m -> m.values().stream())
|
||||
.forEach(v -> assertArrayEquals(value(value), v));
|
||||
i++;
|
||||
}
|
||||
assertEquals(count, i);
|
||||
} catch (IOException e) {
|
||||
fail("Failed due to exception");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -31,20 +31,20 @@ import org.junit.ClassRule;
|
|||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Reruns TestLoadIncrementalHFiles using LoadIncrementalHFiles in secure mode. This suite is unable
|
||||
* Reruns TestBulkLoadHFiles using BulkLoadHFiles 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.
|
||||
* <p>
|
||||
* <p/>
|
||||
* 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({ MiscTests.class, LargeTests.class })
|
||||
public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles {
|
||||
public class TestSecureBulkLoadHFiles extends TestBulkLoadHFiles {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestSecureLoadIncrementalHFiles.class);
|
||||
HBaseClassTestRule.forClass(TestSecureBulkLoadHFiles.class);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
|
@ -53,7 +53,7 @@ public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles {
|
|||
HadoopSecurityEnabledUserProviderForTesting.class);
|
||||
// setup configuration
|
||||
SecureTestUtil.enableSecurity(util.getConfiguration());
|
||||
util.getConfiguration().setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
|
||||
util.getConfiguration().setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
|
||||
MAX_FILES_PER_REGION_PER_FAMILY);
|
||||
// change default behavior so that tag values are returned with normal rpcs
|
||||
util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
|
||||
|
@ -66,5 +66,4 @@ public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles {
|
|||
|
||||
setupNamespace();
|
||||
}
|
||||
|
||||
}
|
|
@ -31,21 +31,20 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Reruns TestSecureLoadIncrementalHFilesSplitRecovery using LoadIncrementalHFiles in secure mode.
|
||||
* Reruns TestBulkLoadHFilesSplitRecovery using BulkLoadHFiles in secure mode.
|
||||
* This suite is unable to verify the security handoff/turnove as miniCluster is running as system
|
||||
* user thus has root privileges and delegation tokens don't seem to work on miniDFS.
|
||||
* <p>
|
||||
* <p/>
|
||||
* 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({ MiscTests.class, LargeTests.class })
|
||||
public class TestSecureLoadIncrementalHFilesSplitRecovery
|
||||
extends TestLoadIncrementalHFilesSplitRecovery {
|
||||
public class TestSecureBulkLoadHFilesSplitRecovery extends TestBulkLoadHFilesSplitRecovery {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestSecureLoadIncrementalHFilesSplitRecovery.class);
|
||||
HBaseClassTestRule.forClass(TestSecureBulkLoadHFilesSplitRecovery.class);
|
||||
|
||||
// This "overrides" the parent static method
|
||||
// make sure they are in sync
|
Loading…
Reference in New Issue