HBASE-19251 Merge RawAsyncTable and AsyncTable

This commit is contained in:
zhangduo 2017-11-16 14:36:28 +08:00
parent 3a4655019d
commit 54827cf613
52 changed files with 1290 additions and 1265 deletions

View File

@ -38,10 +38,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor;
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.RawAsyncTable;
import org.apache.hadoop.hbase.client.RawScanResultConsumer;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
@ -72,14 +72,15 @@ public class AsyncMetaTableAccessor {
private static final Pattern SERVER_COLUMN_PATTERN = Pattern
.compile("^server(_[0-9a-fA-F]{4})?$");
public static CompletableFuture<Boolean> tableExists(RawAsyncTable metaTable, TableName tableName) {
public static CompletableFuture<Boolean> tableExists(AsyncTable<?> metaTable,
TableName tableName) {
if (tableName.equals(META_TABLE_NAME)) {
return CompletableFuture.completedFuture(true);
}
return getTableState(metaTable, tableName).thenApply(Optional::isPresent);
}
public static CompletableFuture<Optional<TableState>> getTableState(RawAsyncTable metaTable,
public static CompletableFuture<Optional<TableState>> getTableState(AsyncTable<?> metaTable,
TableName tableName) {
CompletableFuture<Optional<TableState>> future = new CompletableFuture<>();
Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
@ -110,7 +111,7 @@ public class AsyncMetaTableAccessor {
* @return HRegionLocation for the given region
*/
public static CompletableFuture<Optional<HRegionLocation>> getRegionLocation(
RawAsyncTable metaTable, byte[] regionName) {
AsyncTable<?> metaTable, byte[] regionName) {
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
try {
RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
@ -139,7 +140,7 @@ public class AsyncMetaTableAccessor {
* @return HRegionLocation for the given region
*/
public static CompletableFuture<Optional<HRegionLocation>> getRegionLocationWithEncodedName(
RawAsyncTable metaTable, byte[] encodedRegionName) {
AsyncTable<?> metaTable, byte[] encodedRegionName) {
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
metaTable.scanAll(new Scan().setReadType(ReadType.PREAD).addFamily(HConstants.CATALOG_FAMILY))
.whenComplete(
@ -193,7 +194,7 @@ public class AsyncMetaTableAccessor {
* {@link CompletableFuture}.
*/
public static CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(
RawAsyncTable metaTable, final Optional<TableName> tableName) {
AsyncTable<AdvancedScanResultConsumer> metaTable, Optional<TableName> tableName) {
CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
getTableRegionsAndLocations(metaTable, tableName, true).whenComplete(
(locations, err) -> {
@ -220,7 +221,7 @@ public class AsyncMetaTableAccessor {
* {@link CompletableFuture}.
*/
private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
RawAsyncTable metaTable, final Optional<TableName> tableName,
AsyncTable<AdvancedScanResultConsumer> metaTable, final Optional<TableName> tableName,
final boolean excludeOfflinedSplitParents) {
CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) {
@ -252,7 +253,7 @@ public class AsyncMetaTableAccessor {
}
for (HRegionLocation loc : current.get().getRegionLocations()) {
if (loc != null) {
this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegionInfo(), loc
this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegion(), loc
.getServerName()));
}
}
@ -276,7 +277,7 @@ public class AsyncMetaTableAccessor {
* @param type scanned part of meta
* @param visitor Visitor invoked against each row
*/
private static CompletableFuture<Void> scanMeta(RawAsyncTable metaTable,
private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable,
Optional<TableName> tableName, QueryType type, final Visitor visitor) {
return scanMeta(metaTable, getTableStartRowForMeta(tableName, type),
getTableStopRowForMeta(tableName, type), type, Integer.MAX_VALUE, visitor);
@ -291,8 +292,9 @@ public class AsyncMetaTableAccessor {
* @param maxRows maximum rows to return
* @param visitor Visitor invoked against each row
*/
private static CompletableFuture<Void> scanMeta(RawAsyncTable metaTable, Optional<byte[]> startRow,
Optional<byte[]> stopRow, QueryType type, int maxRows, final Visitor visitor) {
private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable,
Optional<byte[]> startRow, Optional<byte[]> stopRow, QueryType type, int maxRows,
final Visitor visitor) {
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
Scan scan = getMetaScan(metaTable, rowUpperLimit);
for (byte[] family : type.getFamilies()) {
@ -308,11 +310,11 @@ public class AsyncMetaTableAccessor {
}
CompletableFuture<Void> future = new CompletableFuture<Void>();
metaTable.scan(scan, new MetaTableRawScanResultConsumer(rowUpperLimit, visitor, future));
metaTable.scan(scan, new MetaTableScanResultConsumer(rowUpperLimit, visitor, future));
return future;
}
private static final class MetaTableRawScanResultConsumer implements RawScanResultConsumer {
private static final class MetaTableScanResultConsumer implements AdvancedScanResultConsumer {
private int currentRowCount;
@ -322,7 +324,8 @@ public class AsyncMetaTableAccessor {
private final CompletableFuture<Void> future;
MetaTableRawScanResultConsumer(int rowUpperLimit, Visitor visitor, CompletableFuture<Void> future) {
MetaTableScanResultConsumer(int rowUpperLimit, Visitor visitor,
CompletableFuture<Void> future) {
this.rowUpperLimit = rowUpperLimit;
this.visitor = visitor;
this.future = future;
@ -359,7 +362,7 @@ public class AsyncMetaTableAccessor {
}
}
private static Scan getMetaScan(RawAsyncTable metaTable, int rowUpperLimit) {
private static Scan getMetaScan(AsyncTable<?> metaTable, int rowUpperLimit) {
Scan scan = new Scan();
int scannerCaching = metaTable.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);

View File

@ -20,21 +20,26 @@ package org.apache.hadoop.hbase.client;
import java.util.Optional;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
/**
* Receives {@link Result} for an asynchronous scan.
* This is the low level API for asynchronous scan.
* <p>
* Notice that, the {@link #onNext(Result[], ScanController)} method will be called in the thread
* which we send request to HBase service. So if you want the asynchronous scanner fetch data from
* HBase in background while you process the returned data, you need to move the processing work to
* another thread to make the {@code onNext} call return immediately. And please do NOT do any time
* consuming tasks in all methods below unless you know what you are doing.
* All results that match the given scan object will be passed to this class by calling
* {@link #onNext(Result[], ScanController)}. {@link #onComplete()} means the scan is finished, and
* {@link #onError(Throwable)} means we hit an unrecoverable error and the scan is terminated.
* {@link #onHeartbeat(ScanController)} means the RS is still working but we can not get a valid
* result to call {@link #onNext(Result[], ScanController)}. This is usually because the matched
* results are too sparse, for example, a filter which almost filters out everything is specified.
* <p>
* Notice that, all the methods here will be called directly in the thread which we send request to
* HBase service. So if you want the asynchronous scanner fetch data from HBase in background while
* you process the returned data, you need to move the processing work to another thread to make the
* {@link #onNext(Result[], ScanController)} call return immediately. And please do NOT do any time
* consuming tasks in these methods unless you know what you are doing.
* @since 2.0.0
*/
@InterfaceAudience.Public
public interface RawScanResultConsumer {
public interface AdvancedScanResultConsumer extends ScanResultConsumerBase {
/**
* Used to resume a scan.
@ -113,25 +118,4 @@ public interface RawScanResultConsumer {
*/
default void onHeartbeat(ScanController controller) {
}
/**
* Indicate that we hit an unrecoverable error and the scan operation is terminated.
* <p>
* We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
*/
void onError(Throwable error);
/**
* Indicate that the scan operation is completed normally.
*/
void onComplete();
/**
* If {@code scan.isScanMetricsEnabled()} returns true, then this method will be called prior to
* all other methods in this interface to give you the {@link ScanMetrics} instance for this scan
* operation. The {@link ScanMetrics} instance will be updated on-the-fly during the scan, you can
* store it somewhere to get the metrics at any time if you want.
*/
default void onScanMetricsCreated(ScanMetrics scanMetrics) {
}
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.client;
import com.google.protobuf.RpcChannel;
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
@ -33,7 +35,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
@ -42,8 +43,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.RpcChannel;
/**
* The asynchronous administrative API for HBase.
* @since 2.0.0
@ -1072,14 +1071,14 @@ public interface AsyncAdmin {
* </pre>
* @param stubMaker a delegation to the actual {@code newStub} call.
* @param callable a delegation to the actual protobuf rpc call. See the comment of
* {@link CoprocessorCallable} for more details.
* {@link ServiceCaller} for more details.
* @param <S> the type of the asynchronous stub
* @param <R> the type of the return value
* @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
* @see CoprocessorCallable
* @see ServiceCaller
*/
<S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable);
ServiceCaller<S, R> callable);
/**
* Execute the given coprocessor call on the given region server.
@ -1094,15 +1093,15 @@ public interface AsyncAdmin {
* </pre>
* @param stubMaker a delegation to the actual {@code newStub} call.
* @param callable a delegation to the actual protobuf rpc call. See the comment of
* {@link CoprocessorCallable} for more details.
* {@link ServiceCaller} for more details.
* @param serverName the given region server
* @param <S> the type of the asynchronous stub
* @param <R> the type of the return value
* @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
* @see CoprocessorCallable
* @see ServiceCaller
*/
<S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, ServerName serverName);
ServiceCaller<S, R> callable, ServerName serverName);
/**
* List all the dead region servers.

View File

@ -29,12 +29,12 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
class AsyncBufferedMutatorBuilderImpl implements AsyncBufferedMutatorBuilder {
private final AsyncTableBuilder<? extends AsyncTableBase> tableBuilder;
private final AsyncTableBuilder<?> tableBuilder;
private long writeBufferSize;
public AsyncBufferedMutatorBuilderImpl(AsyncConnectionConfiguration connConf,
AsyncTableBuilder<? extends AsyncTableBase> tableBuilder) {
AsyncTableBuilder<?> tableBuilder) {
this.tableBuilder = tableBuilder;
this.writeBufferSize = connConf.getWriteBufferSize();
}

View File

@ -30,12 +30,12 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The implementation of {@link AsyncBufferedMutator}. Simply wrap an {@link AsyncTableBase}.
* The implementation of {@link AsyncBufferedMutator}. Simply wrap an {@link AsyncTable}.
*/
@InterfaceAudience.Private
class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
private final AsyncTableBase table;
private final AsyncTable<?> table;
private final long writeBufferSize;
@ -47,7 +47,7 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
private boolean closed;
AsyncBufferedMutatorImpl(AsyncTableBase table, long writeBufferSize) {
AsyncBufferedMutatorImpl(AsyncTable<?> table, long writeBufferSize) {
this.table = table;
this.writeBufferSize = writeBufferSize;
}

View File

@ -53,7 +53,7 @@ class AsyncClientScanner {
private final ScanMetrics scanMetrics;
private final RawScanResultConsumer consumer;
private final AdvancedScanResultConsumer consumer;
private final TableName tableName;
@ -71,7 +71,7 @@ class AsyncClientScanner {
private final ScanResultCache resultCache;
public AsyncClientScanner(Scan scan, RawScanResultConsumer consumer, TableName tableName,
public AsyncClientScanner(Scan scan, AdvancedScanResultConsumer consumer, TableName tableName,
AsyncConnectionImpl conn, long pauseNs, int maxAttempts, long scanTimeoutNs,
long rpcTimeoutNs, int startLogErrorsCnt) {
if (scan.getStartRow() == null) {

View File

@ -49,32 +49,37 @@ public interface AsyncConnection extends Closeable {
AsyncTableRegionLocator getRegionLocator(TableName tableName);
/**
* Retrieve an {@link RawAsyncTable} implementation for accessing a table.
* Retrieve an {@link AsyncTable} implementation for accessing a table.
* <p>
* The returned instance will use default configs. Use {@link #getRawTableBuilder(TableName)} if you
* want to customize some configs.
* The returned instance will use default configs. Use {@link #getTableBuilder(TableName)} if
* you want to customize some configs.
* <p>
* This method no longer checks table existence. An exception will be thrown if the table does not
* exist only when the first operation is attempted.
* <p>
* The returned {@code CompletableFuture} will be finished directly in the rpc framework's
* callback thread, so typically you should not do any time consuming work inside these methods.
* And also the observer style scan API will use {@link AdvancedScanResultConsumer} which is
* designed for experts only. Only use it when you know what you are doing.
* @param tableName the name of the table
* @return an RawAsyncTable to use for interactions with this table
* @see #getRawTableBuilder(TableName)
* @return an AsyncTable to use for interactions with this table
* @see #getTableBuilder(TableName)
*/
default RawAsyncTable getRawTable(TableName tableName) {
return getRawTableBuilder(tableName).build();
default AsyncTable<AdvancedScanResultConsumer> getTable(TableName tableName) {
return getTableBuilder(tableName).build();
}
/**
* Returns an {@link AsyncTableBuilder} for creating {@link RawAsyncTable}.
* Returns an {@link AsyncTableBuilder} for creating {@link AsyncTable}.
* <p>
* This method no longer checks table existence. An exception will be thrown if the table does not
* exist only when the first operation is attempted.
* @param tableName the name of the table
*/
AsyncTableBuilder<RawAsyncTable> getRawTableBuilder(TableName tableName);
AsyncTableBuilder<AdvancedScanResultConsumer> getTableBuilder(TableName tableName);
/**
* Retrieve an AsyncTable implementation for accessing a table.
* Retrieve an {@link AsyncTable} implementation for accessing a table.
* <p>
* This method no longer checks table existence. An exception will be thrown if the table does not
* exist only when the first operation is attempted.
@ -82,7 +87,7 @@ public interface AsyncConnection extends Closeable {
* @param pool the thread pool to use for executing callback
* @return an AsyncTable to use for interactions with this table
*/
default AsyncTable getTable(TableName tableName, ExecutorService pool) {
default AsyncTable<ScanResultConsumer> getTable(TableName tableName, ExecutorService pool) {
return getTableBuilder(tableName, pool).build();
}
@ -94,7 +99,7 @@ public interface AsyncConnection extends Closeable {
* @param tableName the name of the table
* @param pool the thread pool to use for executing callback
*/
AsyncTableBuilder<AsyncTable> getTableBuilder(TableName tableName, ExecutorService pool);
AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName, ExecutorService pool);
/**
* Retrieve an {@link AsyncAdmin} implementation to administer an HBase cluster.

View File

@ -97,7 +97,7 @@ class AsyncConnectionImpl implements AsyncConnection {
private final AtomicReference<MasterService.Interface> masterStub = new AtomicReference<>();
private final AtomicReference<CompletableFuture<MasterService.Interface>> masterStubMakeFuture =
new AtomicReference<>();
new AtomicReference<>();
public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
User user) {
@ -108,8 +108,8 @@ class AsyncConnectionImpl implements AsyncConnection {
this.rpcClient = RpcClientFactory.createClient(conf, clusterId);
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
this.rpcTimeout = (int) Math.min(Integer.MAX_VALUE,
TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs()));
this.rpcTimeout =
(int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs()));
this.locator = new AsyncRegionLocator(this, RETRY_TIMER);
this.callerFactory = new AsyncRpcRetryingCallerFactory(this, RETRY_TIMER);
if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) {
@ -161,7 +161,7 @@ class AsyncConnectionImpl implements AsyncConnection {
return MasterService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
}
private AdminService.Interface createAdminServerStub(ServerName serverName) throws IOException{
private AdminService.Interface createAdminServerStub(ServerName serverName) throws IOException {
return AdminService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
}
@ -172,38 +172,37 @@ class AsyncConnectionImpl implements AsyncConnection {
}
private void makeMasterStub(CompletableFuture<MasterService.Interface> future) {
registry.getMasterAddress().whenComplete(
(sn, error) -> {
if (sn == null) {
String msg = "ZooKeeper available but no active master location found";
LOG.info(msg);
this.masterStubMakeFuture.getAndSet(null).completeExceptionally(
new MasterNotRunningException(msg));
return;
}
try {
MasterService.Interface stub = createMasterStub(sn);
HBaseRpcController controller = getRpcController();
stub.isMasterRunning(controller, RequestConverter.buildIsMasterRunningRequest(),
new RpcCallback<IsMasterRunningResponse>() {
@Override
public void run(IsMasterRunningResponse resp) {
if (controller.failed() || resp == null
|| (resp != null && !resp.getIsMasterRunning())) {
masterStubMakeFuture.getAndSet(null).completeExceptionally(
new MasterNotRunningException("Master connection is not running anymore"));
} else {
masterStub.set(stub);
masterStubMakeFuture.set(null);
future.complete(stub);
}
registry.getMasterAddress().whenComplete((sn, error) -> {
if (sn == null) {
String msg = "ZooKeeper available but no active master location found";
LOG.info(msg);
this.masterStubMakeFuture.getAndSet(null)
.completeExceptionally(new MasterNotRunningException(msg));
return;
}
try {
MasterService.Interface stub = createMasterStub(sn);
HBaseRpcController controller = getRpcController();
stub.isMasterRunning(controller, RequestConverter.buildIsMasterRunningRequest(),
new RpcCallback<IsMasterRunningResponse>() {
@Override
public void run(IsMasterRunningResponse resp) {
if (controller.failed() || resp == null ||
(resp != null && !resp.getIsMasterRunning())) {
masterStubMakeFuture.getAndSet(null).completeExceptionally(
new MasterNotRunningException("Master connection is not running anymore"));
} else {
masterStub.set(stub);
masterStubMakeFuture.set(null);
future.complete(stub);
}
});
} catch (IOException e) {
this.masterStubMakeFuture.getAndSet(null).completeExceptionally(
new IOException("Failed to create async master stub", e));
}
});
}
});
} catch (IOException e) {
this.masterStubMakeFuture.getAndSet(null)
.completeExceptionally(new IOException("Failed to create async master stub", e));
}
});
}
CompletableFuture<MasterService.Interface> getMasterStub() {
@ -231,8 +230,8 @@ class AsyncConnectionImpl implements AsyncConnection {
new RpcCallback<IsMasterRunningResponse>() {
@Override
public void run(IsMasterRunningResponse resp) {
if (controller.failed() || resp == null
|| (resp != null && !resp.getIsMasterRunning())) {
if (controller.failed() || resp == null ||
(resp != null && !resp.getIsMasterRunning())) {
makeMasterStub(future);
} else {
future.complete(masterStub);
@ -255,22 +254,23 @@ class AsyncConnectionImpl implements AsyncConnection {
}
@Override
public AsyncTableBuilder<RawAsyncTable> getRawTableBuilder(TableName tableName) {
return new AsyncTableBuilderBase<RawAsyncTable>(tableName, connConf) {
public AsyncTableBuilder<AdvancedScanResultConsumer> getTableBuilder(TableName tableName) {
return new AsyncTableBuilderBase<AdvancedScanResultConsumer>(tableName, connConf) {
@Override
public RawAsyncTable build() {
public AsyncTable<AdvancedScanResultConsumer> build() {
return new RawAsyncTableImpl(AsyncConnectionImpl.this, this);
}
};
}
@Override
public AsyncTableBuilder<AsyncTable> getTableBuilder(TableName tableName, ExecutorService pool) {
return new AsyncTableBuilderBase<AsyncTable>(tableName, connConf) {
public AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName,
ExecutorService pool) {
return new AsyncTableBuilderBase<ScanResultConsumer>(tableName, connConf) {
@Override
public AsyncTable build() {
public AsyncTable<ScanResultConsumer> build() {
RawAsyncTableImpl rawTable = new RawAsyncTableImpl(AsyncConnectionImpl.this, this);
return new AsyncTableImpl(AsyncConnectionImpl.this, rawTable, pool);
}
@ -293,7 +293,7 @@ class AsyncConnectionImpl implements AsyncConnection {
@Override
public AsyncAdmin build() {
RawAsyncHBaseAdmin rawAdmin =
new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
return new AsyncHBaseAdmin(rawAdmin, pool);
}
};
@ -301,7 +301,7 @@ class AsyncConnectionImpl implements AsyncConnection {
@Override
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {
return new AsyncBufferedMutatorBuilderImpl(connConf, getRawTableBuilder(tableName));
return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName));
}
@Override

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
@ -56,7 +55,7 @@ import org.apache.yetus.audience.InterfaceAudience;
* @see AsyncConnection#getAdminBuilder(ExecutorService)
*/
@InterfaceAudience.Private
public class AsyncHBaseAdmin implements AsyncAdmin {
class AsyncHBaseAdmin implements AsyncAdmin {
private final RawAsyncHBaseAdmin rawAdmin;
@ -705,13 +704,13 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
@Override
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable) {
ServiceCaller<S, R> callable) {
return wrap(rawAdmin.coprocessorService(stubMaker, callable));
}
@Override
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, ServerName serverName) {
ServiceCaller<S, R> callable, ServerName serverName) {
return wrap(rawAdmin.coprocessorService(stubMaker, callable, serverName));
}

View File

@ -20,10 +20,10 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.HConstants.CATALOG_FAMILY;
import static org.apache.hadoop.hbase.HConstants.NINES;
import static org.apache.hadoop.hbase.HConstants.ZEROES;
import static org.apache.hadoop.hbase.HRegionInfo.createRegionName;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
import static org.apache.hadoop.hbase.client.RegionInfo.createRegionName;
import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
@ -45,14 +45,13 @@ import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The asynchronous locator for regions other than meta.
@ -63,7 +62,7 @@ class AsyncNonMetaRegionLocator {
private static final Log LOG = LogFactory.getLog(AsyncNonMetaRegionLocator.class);
static final String MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE =
"hbase.client.meta.max.concurrent.locate.per.table";
"hbase.client.meta.max.concurrent.locate.per.table";
private static final int DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE = 8;
@ -102,12 +101,12 @@ class AsyncNonMetaRegionLocator {
private static final class TableCache {
public final ConcurrentNavigableMap<byte[], HRegionLocation> cache =
new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
public final Set<LocateRequest> pendingRequests = new HashSet<>();
public final Map<LocateRequest, CompletableFuture<HRegionLocation>> allRequests =
new LinkedHashMap<>();
new LinkedHashMap<>();
public boolean hasQuota(int max) {
return pendingRequests.size() < max;
@ -126,8 +125,8 @@ class AsyncNonMetaRegionLocator {
}
public void clearCompletedRequests(Optional<HRegionLocation> location) {
for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter = allRequests
.entrySet().iterator(); iter.hasNext();) {
for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter =
allRequests.entrySet().iterator(); iter.hasNext();) {
Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>> entry = iter.next();
if (tryComplete(entry.getKey(), entry.getValue(), location)) {
iter.remove();
@ -146,15 +145,16 @@ class AsyncNonMetaRegionLocator {
HRegionLocation loc = location.get();
boolean completed;
if (req.locateType.equals(RegionLocateType.BEFORE)) {
// for locating the row before current row, the common case is to find the previous region in
// reverse scan, so we check the endKey first. In general, the condition should be startKey <
// req.row and endKey >= req.row. Here we split it to endKey == req.row || (endKey > req.row
// && startKey < req.row). The two conditions are equal since startKey < endKey.
int c = Bytes.compareTo(loc.getRegionInfo().getEndKey(), req.row);
// for locating the row before current row, the common case is to find the previous region
// in reverse scan, so we check the endKey first. In general, the condition should be
// startKey < req.row and endKey >= req.row. Here we split it to endKey == req.row ||
// (endKey > req.row && startKey < req.row). The two conditions are equal since startKey <
// endKey.
int c = Bytes.compareTo(loc.getRegion().getEndKey(), req.row);
completed =
c == 0 || (c > 0 && Bytes.compareTo(loc.getRegionInfo().getStartKey(), req.row) < 0);
c == 0 || (c > 0 && Bytes.compareTo(loc.getRegion().getStartKey(), req.row) < 0);
} else {
completed = loc.getRegionInfo().containsRow(req.row);
completed = loc.getRegion().containsRow(req.row);
}
if (completed) {
future.complete(loc);
@ -176,13 +176,13 @@ class AsyncNonMetaRegionLocator {
}
private void removeFromCache(HRegionLocation loc) {
TableCache tableCache = cache.get(loc.getRegionInfo().getTable());
TableCache tableCache = cache.get(loc.getRegion().getTable());
if (tableCache == null) {
return;
}
tableCache.cache.computeIfPresent(loc.getRegionInfo().getStartKey(), (k, oldLoc) -> {
tableCache.cache.computeIfPresent(loc.getRegion().getStartKey(), (k, oldLoc) -> {
if (oldLoc.getSeqNum() > loc.getSeqNum() ||
!oldLoc.getServerName().equals(loc.getServerName())) {
!oldLoc.getServerName().equals(loc.getServerName())) {
return oldLoc;
}
return null;
@ -194,16 +194,16 @@ class AsyncNonMetaRegionLocator {
if (LOG.isTraceEnabled()) {
LOG.trace("Try adding " + loc + " to cache");
}
byte[] startKey = loc.getRegionInfo().getStartKey();
byte[] startKey = loc.getRegion().getStartKey();
HRegionLocation oldLoc = tableCache.cache.putIfAbsent(startKey, loc);
if (oldLoc == null) {
return true;
}
if (oldLoc.getSeqNum() > loc.getSeqNum() ||
oldLoc.getServerName().equals(loc.getServerName())) {
oldLoc.getServerName().equals(loc.getServerName())) {
if (LOG.isTraceEnabled()) {
LOG.trace("Will not add " + loc + " to cache because the old value " + oldLoc +
" is newer than us or has the same server name");
" is newer than us or has the same server name");
}
return false;
}
@ -213,8 +213,8 @@ class AsyncNonMetaRegionLocator {
}
if (LOG.isTraceEnabled()) {
LOG.trace("Will not add " + loc + " to cache because the old value " + oldValue +
" is newer than us or has the same server name." +
" Maybe it is updated before we replace it");
" is newer than us or has the same server name." +
" Maybe it is updated before we replace it");
}
return oldValue;
});
@ -223,7 +223,7 @@ class AsyncNonMetaRegionLocator {
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
justification = "Called by lambda expression")
private void addToCache(HRegionLocation loc) {
addToCache(getTableCache(loc.getRegionInfo().getTable()), loc);
addToCache(getTableCache(loc.getRegion().getTable()), loc);
if (LOG.isTraceEnabled()) {
LOG.trace("Try adding " + loc + " to cache");
}
@ -232,9 +232,8 @@ class AsyncNonMetaRegionLocator {
private void complete(TableName tableName, LocateRequest req, HRegionLocation loc,
Throwable error) {
if (error != null) {
LOG.warn(
"Failed to locate region in '" + tableName + "', row='" + Bytes.toStringBinary(req.row)
+ "', locateType=" + req.locateType, error);
LOG.warn("Failed to locate region in '" + tableName + "', row='" +
Bytes.toStringBinary(req.row) + "', locateType=" + req.locateType, error);
}
Optional<LocateRequest> toSend = Optional.empty();
TableCache tableCache = getTableCache(tableName);
@ -283,7 +282,7 @@ class AsyncNonMetaRegionLocator {
RegionLocations locs = MetaTableAccessor.getRegionLocations(results.get(0));
if (LOG.isDebugEnabled()) {
LOG.debug("The fetched location of '" + tableName + "', row='" +
Bytes.toStringBinary(req.row) + "', locateType=" + req.locateType + " is " + locs);
Bytes.toStringBinary(req.row) + "', locateType=" + req.locateType + " is " + locs);
}
if (locs == null || locs.getDefaultRegionLocation() == null) {
complete(tableName, req, null,
@ -292,7 +291,7 @@ class AsyncNonMetaRegionLocator {
return;
}
HRegionLocation loc = locs.getDefaultRegionLocation();
HRegionInfo info = loc.getRegionInfo();
RegionInfo info = loc.getRegion();
if (info == null) {
complete(tableName, req, null,
new IOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
@ -308,12 +307,12 @@ class AsyncNonMetaRegionLocator {
complete(tableName, req, null,
new RegionOfflineException(
"the only available region for the required row is a split parent," +
" the daughters should be online soon: '" + info.getRegionNameAsString() + "'"));
" the daughters should be online soon: '" + info.getRegionNameAsString() + "'"));
return;
}
if (info.isOffline()) {
complete(tableName, req, null, new RegionOfflineException("the region is offline, could" +
" be caused by a disable table call: '" + info.getRegionNameAsString() + "'"));
" be caused by a disable table call: '" + info.getRegionNameAsString() + "'"));
return;
}
if (loc.getServerName() == null) {
@ -332,11 +331,11 @@ class AsyncNonMetaRegionLocator {
return null;
}
HRegionLocation loc = entry.getValue();
byte[] endKey = loc.getRegionInfo().getEndKey();
byte[] endKey = loc.getRegion().getEndKey();
if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
if (LOG.isTraceEnabled()) {
LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.CURRENT);
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.CURRENT);
}
return loc;
} else {
@ -347,16 +346,16 @@ class AsyncNonMetaRegionLocator {
private HRegionLocation locateRowBeforeInCache(TableCache tableCache, TableName tableName,
byte[] row) {
Map.Entry<byte[], HRegionLocation> entry =
isEmptyStopRow(row) ? tableCache.cache.lastEntry() : tableCache.cache.lowerEntry(row);
isEmptyStopRow(row) ? tableCache.cache.lastEntry() : tableCache.cache.lowerEntry(row);
if (entry == null) {
return null;
}
HRegionLocation loc = entry.getValue();
if (isEmptyStopRow(loc.getRegionInfo().getEndKey()) ||
Bytes.compareTo(loc.getRegionInfo().getEndKey(), row) >= 0) {
if (isEmptyStopRow(loc.getRegion().getEndKey()) ||
Bytes.compareTo(loc.getRegion().getEndKey(), row) >= 0) {
if (LOG.isTraceEnabled()) {
LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.BEFORE);
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.BEFORE);
}
return loc;
} else {
@ -367,7 +366,7 @@ class AsyncNonMetaRegionLocator {
private void locateInMeta(TableName tableName, LocateRequest req) {
if (LOG.isTraceEnabled()) {
LOG.trace("Try locate '" + tableName + "', row='" + Bytes.toStringBinary(req.row) +
"', locateType=" + req.locateType + " in meta");
"', locateType=" + req.locateType + " in meta");
}
byte[] metaKey;
if (req.locateType.equals(RegionLocateType.BEFORE)) {
@ -380,7 +379,7 @@ class AsyncNonMetaRegionLocator {
} else {
metaKey = createRegionName(tableName, req.row, NINES, false);
}
conn.getRawTable(META_TABLE_NAME)
conn.getTable(META_TABLE_NAME)
.scanAll(new Scan().withStartRow(metaKey).setReversed(true).addFamily(CATALOG_FAMILY)
.setOneRowLimit())
.whenComplete((results, error) -> onScanComplete(tableName, req, results, error));
@ -389,8 +388,8 @@ class AsyncNonMetaRegionLocator {
private HRegionLocation locateInCache(TableCache tableCache, TableName tableName, byte[] row,
RegionLocateType locateType) {
return locateType.equals(RegionLocateType.BEFORE)
? locateRowBeforeInCache(tableCache, tableName, row)
: locateRowInCache(tableCache, tableName, row);
? locateRowBeforeInCache(tableCache, tableName, row)
: locateRowInCache(tableCache, tableName, row);
}
// locateToPrevious is true means we will use the start key of a region to locate the region
@ -451,11 +450,11 @@ class AsyncNonMetaRegionLocator {
void updateCachedLocation(HRegionLocation loc, Throwable exception) {
AsyncRegionLocator.updateCachedLocation(loc, exception, l -> {
TableCache tableCache = cache.get(l.getRegionInfo().getTable());
TableCache tableCache = cache.get(l.getRegion().getTable());
if (tableCache == null) {
return null;
}
return tableCache.cache.get(l.getRegionInfo().getStartKey());
return tableCache.cache.get(l.getRegion().getStartKey());
}, this::addToCache, this::removeFromCache);
}

View File

@ -153,7 +153,7 @@ class AsyncRpcRetryingCallerFactory {
private ScanResultCache resultCache;
private RawScanResultConsumer consumer;
private AdvancedScanResultConsumer consumer;
private ClientService.Interface stub;
@ -192,7 +192,7 @@ class AsyncRpcRetryingCallerFactory {
return this;
}
public ScanSingleRegionCallerBuilder consumer(RawScanResultConsumer consumer) {
public ScanSingleRegionCallerBuilder consumer(AdvancedScanResultConsumer consumer) {
this.consumer = consumer;
return this;
}

View File

@ -28,14 +28,8 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
import static org.apache.hadoop.hbase.client.ConnectionUtils.updateResultsMetrics;
import static org.apache.hadoop.hbase.client.ConnectionUtils.updateServerSideMetrics;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
@ -47,13 +41,18 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RawScanResultConsumer.ScanResumer;
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer.ScanResumer;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@ -61,7 +60,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientServ
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.Interface;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Retry caller for scanning a region.
@ -84,7 +82,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
private final ScanResultCache resultCache;
private final RawScanResultConsumer consumer;
private final AdvancedScanResultConsumer consumer;
private final ClientService.Interface stub;
@ -143,7 +141,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
// Notice that, the public methods of this class is supposed to be called by upper layer only, and
// package private methods can only be called within the implementation of
// AsyncScanSingleRegionRpcRetryingCaller.
private final class ScanControllerImpl implements RawScanResultConsumer.ScanController {
private final class ScanControllerImpl implements AdvancedScanResultConsumer.ScanController {
// Make sure the methods are only called in this thread.
private final Thread callerThread;
@ -217,7 +215,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
// Notice that, the public methods of this class is supposed to be called by upper layer only, and
// package private methods can only be called within the implementation of
// AsyncScanSingleRegionRpcRetryingCaller.
private final class ScanResumerImpl implements RawScanResultConsumer.ScanResumer {
private final class ScanResumerImpl implements AdvancedScanResultConsumer.ScanResumer {
// INITIALIZED -> SUSPENDED -> RESUMED
// INITIALIZED -> RESUMED
@ -301,7 +299,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
public AsyncScanSingleRegionRpcRetryingCaller(HashedWheelTimer retryTimer,
AsyncConnectionImpl conn, Scan scan, ScanMetrics scanMetrics, long scannerId,
ScanResultCache resultCache, RawScanResultConsumer consumer, Interface stub,
ScanResultCache resultCache, AdvancedScanResultConsumer consumer, Interface stub,
HRegionLocation loc, boolean isRegionServerRemote, long scannerLeaseTimeoutPeriodNs,
long pauseNs, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) {
this.retryTimer = retryTimer;
@ -344,8 +342,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
stub.scan(controller, req, resp -> {
if (controller.failed()) {
LOG.warn("Call to " + loc.getServerName() + " for closing scanner id = " + scannerId +
" for " + loc.getRegionInfo().getEncodedName() + " of " +
loc.getRegionInfo().getTable() + " failed, ignore, probably already closed",
" for " + loc.getRegion().getEncodedName() + " of " +
loc.getRegion().getTable() + " failed, ignore, probably already closed",
controller.getFailed());
}
});
@ -384,7 +382,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
error = translateException(error);
if (tries > startLogErrorsCnt) {
LOG.warn("Call to " + loc.getServerName() + " for scanner id = " + scannerId + " for " +
loc.getRegionInfo().getEncodedName() + " of " + loc.getRegionInfo().getTable() +
loc.getRegion().getEncodedName() + " of " + loc.getRegion().getTable() +
" failed, , tries = " + tries + ", maxAttempts = " + maxAttempts + ", timeout = " +
TimeUnit.NANOSECONDS.toMillis(scanTimeoutNs) + " ms, time elapsed = " + elapsedMs() +
" ms",
@ -433,18 +431,18 @@ class AsyncScanSingleRegionRpcRetryingCaller {
}
private void completeWhenNoMoreResultsInRegion() {
if (noMoreResultsForScan(scan, loc.getRegionInfo())) {
if (noMoreResultsForScan(scan, loc.getRegion())) {
completeNoMoreResults();
} else {
completeWithNextStartRow(loc.getRegionInfo().getEndKey(), true);
completeWithNextStartRow(loc.getRegion().getEndKey(), true);
}
}
private void completeReversedWhenNoMoreResultsInRegion() {
if (noMoreResultsForReverseScan(scan, loc.getRegionInfo())) {
if (noMoreResultsForReverseScan(scan, loc.getRegion())) {
completeNoMoreResults();
} else {
completeWithNextStartRow(loc.getRegionInfo().getStartKey(), false);
completeWithNextStartRow(loc.getRegion().getStartKey(), false);
}
}

View File

@ -68,7 +68,7 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
} catch (IOException e) {
onError(e,
() -> "Get async stub to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row)
+ "' in " + loc.getRegionInfo().getEncodedName() + " of " + tableName + " failed",
+ "' in " + loc.getRegion().getEncodedName() + " of " + tableName + " failed",
err -> conn.getLocator().updateCachedLocation(loc, err));
return;
}
@ -78,7 +78,7 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
if (error != null) {
onError(error,
() -> "Call to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) + "' in "
+ loc.getRegionInfo().getEncodedName() + " of " + tableName + " failed",
+ loc.getRegion().getEncodedName() + " of " + tableName + " failed",
err -> conn.getLocator().updateCachedLocation(loc, err));
return;
}

View File

@ -17,20 +17,277 @@
*/
package org.apache.hadoop.hbase.client;
import static java.util.stream.Collectors.toList;
import static org.apache.hadoop.hbase.client.ConnectionUtils.allOf;
import static org.apache.hadoop.hbase.client.ConnectionUtils.toCheckExistenceOnly;
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.CompareOperator;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
/**
* The asynchronous table for normal users.
* The interface for asynchronous version of Table. Obtain an instance from a
* {@link AsyncConnection}.
* <p>
* The implementation is required to be thread safe.
* <p>
* The implementation should make sure that user can do everything they want to the returned
* {@code CompletableFuture} without breaking anything. Usually the implementation will require user
* to provide a {@code ExecutorService}.
* Usually the implementation will not throw any exception directly. You need to get the exception
* from the returned {@link CompletableFuture}.
* @since 2.0.0
*/
@InterfaceAudience.Public
public interface AsyncTable extends AsyncTableBase {
public interface AsyncTable<C extends ScanResultConsumerBase> {
/**
* Gets the fully qualified table name instance of this table.
*/
TableName getName();
/**
* Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
* <p>
* The reference returned is not a copy, so any change made to it will affect this instance.
*/
Configuration getConfiguration();
/**
* Get timeout of each rpc request in this Table instance. It will be overridden by a more
* specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
* @see #getReadRpcTimeout(TimeUnit)
* @see #getWriteRpcTimeout(TimeUnit)
* @param unit the unit of time the timeout to be represented in
* @return rpc timeout in the specified time unit
*/
long getRpcTimeout(TimeUnit unit);
/**
* Get timeout of each rpc read request in this Table instance.
* @param unit the unit of time the timeout to be represented in
* @return read rpc timeout in the specified time unit
*/
long getReadRpcTimeout(TimeUnit unit);
/**
* Get timeout of each rpc write request in this Table instance.
* @param unit the unit of time the timeout to be represented in
* @return write rpc timeout in the specified time unit
*/
long getWriteRpcTimeout(TimeUnit unit);
/**
* Get timeout of each operation in Table instance.
* @param unit the unit of time the timeout to be represented in
* @return operation rpc timeout in the specified time unit
*/
long getOperationTimeout(TimeUnit unit);
/**
* Get the timeout of a single operation in a scan. It works like operation timeout for other
* operations.
* @param unit the unit of time the timeout to be represented in
* @return scan rpc timeout in the specified time unit
*/
long getScanTimeout(TimeUnit unit);
/**
* Test for the existence of columns in the table, as specified by the Get.
* <p>
* This will return true if the Get matches one or more keys, false if not.
* <p>
* This is a server-side call so it prevents any data from being transfered to the client.
* @return true if the specified Get matches one or more keys, false if not. The return value will
* be wrapped by a {@link CompletableFuture}.
*/
default CompletableFuture<Boolean> exists(Get get) {
return get(toCheckExistenceOnly(get)).thenApply(r -> r.getExists());
}
/**
* Extracts certain cells from a given row.
* @param get The object that specifies what data to fetch and from which row.
* @return The data coming from the specified row, if it exists. If the row specified doesn't
* exist, the {@link Result} instance returned won't contain any
* {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}. The
* return value will be wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Result> get(Get get);
/**
* Puts some data to the table.
* @param put The data to put.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
CompletableFuture<Void> put(Put put);
/**
* Deletes the specified cells/row.
* @param delete The object that specifies what to delete.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
CompletableFuture<Void> delete(Delete delete);
/**
* Appends values to one or more columns within a single row.
* <p>
* This operation does not appear atomic to readers. Appends are done under a single row lock, so
* write operations to a row are synchronized, but readers do not take row locks so get and scan
* operations can see this operation partially completed.
* @param append object that specifies the columns and amounts to be used for the increment
* operations
* @return values of columns after the append operation (maybe null). The return value will be
* wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Result> append(Append append);
/**
* Increments one or more columns within a single row.
* <p>
* This operation does not appear atomic to readers. Increments are done under a single row lock,
* so write operations to a row are synchronized, but readers do not take row locks so get and
* scan operations can see this operation partially completed.
* @param increment object that specifies the columns and amounts to be used for the increment
* operations
* @return values of columns after the increment. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
CompletableFuture<Result> increment(Increment increment);
/**
* See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
* <p>
* The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
* @param row The row that contains the cell to increment.
* @param family The column family of the cell to increment.
* @param qualifier The column qualifier of the cell to increment.
* @param amount The amount to increment the cell with (or decrement, if the amount is negative).
* @return The new value, post increment. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
long amount) {
return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
}
/**
* Atomically increments a column value. If the column value already exists and is not a
* big-endian long, this could throw an exception. If the column value does not yet exist it is
* initialized to <code>amount</code> and written to the specified column.
* <p>
* Setting durability to {@link Durability#SKIP_WAL} means that in a fail scenario you will lose
* any increments that have not been flushed.
* @param row The row that contains the cell to increment.
* @param family The column family of the cell to increment.
* @param qualifier The column qualifier of the cell to increment.
* @param amount The amount to increment the cell with (or decrement, if the amount is negative).
* @param durability The persistence guarantee for this increment.
* @return The new value, post increment. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
long amount, Durability durability) {
Preconditions.checkNotNull(row, "row is null");
Preconditions.checkNotNull(family, "family is null");
return increment(
new Increment(row).addColumn(family, qualifier, amount).setDurability(durability))
.thenApply(r -> Bytes.toLong(r.getValue(family, qualifier)));
}
/**
* Atomically checks if a row/family/qualifier value matches the expected value. If it does, it
* adds the Put/Delete/RowMutations.
* <p>
* Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it.
* This is a fluent style API, the code is like:
*
* <pre>
* <code>
* table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put)
* .thenAccept(succ -> {
* if (succ) {
* System.out.println("Check and put succeeded");
* } else {
* System.out.println("Check and put failed");
* }
* });
* </code>
* </pre>
*/
CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family);
/**
* A helper class for sending checkAndMutate request.
*/
interface CheckAndMutateBuilder {
/**
* @param qualifier column qualifier to check.
*/
CheckAndMutateBuilder qualifier(byte[] qualifier);
/**
* Check for lack of column.
*/
CheckAndMutateBuilder ifNotExists();
default CheckAndMutateBuilder ifEquals(byte[] value) {
return ifMatches(CompareOperator.EQUAL, value);
}
/**
* @param compareOp comparison operator to use
* @param value the expected value
*/
CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
/**
* @param put data to put if check succeeds
* @return {@code true} if the new put was executed, {@code false} otherwise. The return value
* will be wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Boolean> thenPut(Put put);
/**
* @param delete data to delete if check succeeds
* @return {@code true} if the new delete was executed, {@code false} otherwise. The return
* value will be wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Boolean> thenDelete(Delete delete);
/**
* @param mutation mutations to perform if check succeeds
* @return true if the new mutation was executed, false otherwise. The return value will be
* wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Boolean> thenMutate(RowMutations mutation);
}
/**
* Performs multiple mutations atomically on a single row. Currently {@link Put} and
* {@link Delete} are supported.
* @param mutation object that specifies the set of mutations to perform atomically
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
CompletableFuture<Void> mutateRow(RowMutations mutation);
/**
* The scan API uses the observer pattern.
* @param scan A configured {@link Scan} object.
* @param consumer the consumer used to receive results.
* @see ScanResultConsumer
* @see AdvancedScanResultConsumer
*/
void scan(Scan scan, C consumer);
/**
* Gets a scanner on the current table for the given family.
@ -59,13 +316,300 @@ public interface AsyncTable extends AsyncTableBase {
ResultScanner getScanner(Scan scan);
/**
* The scan API uses the observer pattern. All results that match the given scan object will be
* passed to the given {@code consumer} by calling {@link ScanResultConsumer#onNext(Result)}.
* {@link ScanResultConsumer#onComplete()} means the scan is finished, and
* {@link ScanResultConsumer#onError(Throwable)} means we hit an unrecoverable error and the scan
* is terminated.
* @param scan A configured {@link Scan} object.
* @param consumer the consumer used to receive results.
* Return all the results that match the given scan object.
* <p>
* Notice that usually you should use this method with a {@link Scan} object that has limit set.
* For example, if you want to get the closest row after a given row, you could do this:
* <p>
*
* <pre>
* <code>
* table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -> {
* if (results.isEmpty()) {
* System.out.println("No row after " + Bytes.toStringBinary(row));
* } else {
* System.out.println("The closest row after " + Bytes.toStringBinary(row) + " is "
* + Bytes.toStringBinary(results.stream().findFirst().get().getRow()));
* }
* });
* </code>
* </pre>
* <p>
* If your result set is very large, you should use other scan method to get a scanner or use
* callback to process the results. They will do chunking to prevent OOM. The scanAll method will
* fetch all the results and store them in a List and then return the list to you.
* <p>
* The scan metrics will be collected background if you enable it but you have no way to get it.
* Usually you can get scan metrics from {@code ResultScanner}, or through
* {@code ScanResultConsumer.onScanMetricsCreated} but this method only returns a list of results.
* So if you really care about scan metrics then you'd better use other scan methods which return
* a {@code ResultScanner} or let you pass in a {@code ScanResultConsumer}. There is no
* performance difference between these scan methods so do not worry.
* @param scan A configured {@link Scan} object. So if you use this method to fetch a really large
* result set, it is likely to cause OOM.
* @return The results of this small scan operation. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
void scan(Scan scan, ScanResultConsumer consumer);
CompletableFuture<List<Result>> scanAll(Scan scan);
/**
* Test for the existence of columns in the table, as specified by the Gets.
* <p>
* This will return a list of booleans. Each value will be true if the related Get matches one or
* more keys, false if not.
* <p>
* This is a server-side call so it prevents any data from being transferred to the client.
* @param gets the Gets
* @return A list of {@link CompletableFuture}s that represent the existence for each get.
*/
default List<CompletableFuture<Boolean>> exists(List<Get> gets) {
return get(toCheckExistenceOnly(gets)).stream()
.<CompletableFuture<Boolean>> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
}
/**
* A simple version for batch exists. It will fail if there are any failures and you will get the
* whole result boolean list at once if the operation is succeeded.
* @param gets the Gets
* @return A {@link CompletableFuture} that wrapper the result boolean list.
*/
default CompletableFuture<List<Boolean>> existsAll(List<Get> gets) {
return allOf(exists(gets));
}
/**
* Extracts certain cells from the given rows, in batch.
* <p>
* Notice that you may not get all the results with this function, which means some of the
* returned {@link CompletableFuture}s may succeed while some of the other returned
* {@link CompletableFuture}s may fail.
* @param gets The objects that specify what data to fetch and from which rows.
* @return A list of {@link CompletableFuture}s that represent the result for each get.
*/
List<CompletableFuture<Result>> get(List<Get> gets);
/**
* A simple version for batch get. It will fail if there are any failures and you will get the
* whole result list at once if the operation is succeeded.
* @param gets The objects that specify what data to fetch and from which rows.
* @return A {@link CompletableFuture} that wrapper the result list.
*/
default CompletableFuture<List<Result>> getAll(List<Get> gets) {
return allOf(get(gets));
}
/**
* Puts some data in the table, in batch.
* @param puts The list of mutations to apply.
* @return A list of {@link CompletableFuture}s that represent the result for each put.
*/
List<CompletableFuture<Void>> put(List<Put> puts);
/**
* A simple version of batch put. It will fail if there are any failures.
* @param puts The list of mutations to apply.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
default CompletableFuture<Void> putAll(List<Put> puts) {
return allOf(put(puts)).thenApply(r -> null);
}
/**
* Deletes the specified cells/rows in bulk.
* @param deletes list of things to delete.
* @return A list of {@link CompletableFuture}s that represent the result for each delete.
*/
List<CompletableFuture<Void>> delete(List<Delete> deletes);
/**
* A simple version of batch delete. It will fail if there are any failures.
* @param deletes list of things to delete.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
default CompletableFuture<Void> deleteAll(List<Delete> deletes) {
return allOf(delete(deletes)).thenApply(r -> null);
}
/**
* Method that does a batch call on Deletes, Gets, Puts, Increments and Appends. The ordering of
* execution of the actions is not defined. Meaning if you do a Put and a Get in the same
* {@link #batch} call, you will not necessarily be guaranteed that the Get returns what the Put
* had put.
* @param actions list of Get, Put, Delete, Increment, Append objects
* @return A list of {@link CompletableFuture}s that represent the result for each action.
*/
<T> List<CompletableFuture<T>> batch(List<? extends Row> actions);
/**
* A simple version of batch. It will fail if there are any failures and you will get the whole
* result list at once if the operation is succeeded.
* @param actions list of Get, Put, Delete, Increment, Append objects
* @return A list of the result for the actions. Wrapped by a {@link CompletableFuture}.
*/
default <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
return allOf(batch(actions));
}
/**
* Execute the given coprocessor call on the region which contains the given {@code row}.
* <p>
* The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
* one line lambda expression, like:
*
* <pre>
* <code>
* channel -> xxxService.newStub(channel)
* </code>
* </pre>
*
* @param stubMaker a delegation to the actual {@code newStub} call.
* @param callable a delegation to the actual protobuf rpc call. See the comment of
* {@link ServiceCaller} for more details.
* @param row The row key used to identify the remote region location
* @param <S> the type of the asynchronous stub
* @param <R> the type of the return value
* @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
* @see ServiceCaller
*/
<S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, byte[] row);
/**
* The callback when we want to execute a coprocessor call on a range of regions.
* <p>
* As the locating itself also takes some time, the implementation may want to send rpc calls on
* the fly, which means we do not know how many regions we have when we get the return value of
* the rpc calls, so we need an {@link #onComplete()} which is used to tell you that we have
* passed all the return values to you(through the {@link #onRegionComplete(RegionInfo, Object)}
* or {@link #onRegionError(RegionInfo, Throwable)} calls), i.e, there will be no
* {@link #onRegionComplete(RegionInfo, Object)} or {@link #onRegionError(RegionInfo, Throwable)}
* calls in the future.
* <p>
* Here is a pseudo code to describe a typical implementation of a range coprocessor service
* method to help you better understand how the {@link CoprocessorCallback} will be called. The
* {@code callback} in the pseudo code is our {@link CoprocessorCallback}. And notice that the
* {@code whenComplete} is {@code CompletableFuture.whenComplete}.
*
* <pre>
* locateThenCall(byte[] row) {
* locate(row).whenComplete((location, locateError) -> {
* if (locateError != null) {
* callback.onError(locateError);
* return;
* }
* incPendingCall();
* region = location.getRegion();
* if (region.getEndKey() > endKey) {
* locateEnd = true;
* } else {
* locateThenCall(region.getEndKey());
* }
* sendCall().whenComplete((resp, error) -> {
* if (error != null) {
* callback.onRegionError(region, error);
* } else {
* callback.onRegionComplete(region, resp);
* }
* if (locateEnd && decPendingCallAndGet() == 0) {
* callback.onComplete();
* }
* });
* });
* }
* </pre>
*/
@InterfaceAudience.Public
interface CoprocessorCallback<R> {
/**
* @param region the region that the response belongs to
* @param resp the response of the coprocessor call
*/
void onRegionComplete(RegionInfo region, R resp);
/**
* @param region the region that the error belongs to
* @param error the response error of the coprocessor call
*/
void onRegionError(RegionInfo region, Throwable error);
/**
* Indicate that all responses of the regions have been notified by calling
* {@link #onRegionComplete(RegionInfo, Object)} or
* {@link #onRegionError(RegionInfo, Throwable)}.
*/
void onComplete();
/**
* Indicate that we got an error which does not belong to any regions. Usually a locating error.
*/
void onError(Throwable error);
}
/**
* Helper class for sending coprocessorService request that executes a coprocessor call on regions
* which are covered by a range.
* <p>
* If {@code fromRow} is not specified the selection will start with the first table region. If
* {@code toRow} is not specified the selection will continue through the last table region.
* @param <S> the type of the protobuf Service you want to call.
* @param <R> the type of the return value.
*/
interface CoprocessorServiceBuilder<S, R> {
/**
* @param startKey start region selection with region containing this row, inclusive.
*/
default CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey) {
return fromRow(startKey, true);
}
/**
* @param startKey start region selection with region containing this row
* @param inclusive whether to include the startKey
*/
CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey, boolean inclusive);
/**
* @param endKey select regions up to and including the region containing this row, exclusive.
*/
default CoprocessorServiceBuilder<S, R> toRow(byte[] endKey) {
return toRow(endKey, false);
}
/**
* @param endKey select regions up to and including the region containing this row
* @param inclusive whether to include the endKey
*/
CoprocessorServiceBuilder<S, R> toRow(byte[] endKey, boolean inclusive);
/**
* Execute the coprocessorService request. You can get the response through the
* {@link CoprocessorCallback}.
*/
void execute();
}
/**
* Execute a coprocessor call on the regions which are covered by a range.
* <p>
* Use the returned {@link CoprocessorServiceBuilder} construct your request and then execute it.
* <p>
* The {@code stubMaker} is just a delegation to the {@code xxxService.newStub} call. Usually it
* is only a one line lambda expression, like:
*
* <pre>
* <code>
* channel -> xxxService.newStub(channel)
* </code>
* </pre>
*
* @param stubMaker a delegation to the actual {@code newStub} call.
* @param callable a delegation to the actual protobuf rpc call. See the comment of
* {@link ServiceCaller} for more details.
* @param callback callback to get the response. See the comment of {@link CoprocessorCallback}
* for more details.
*/
<S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, CoprocessorCallback<R> callback);
}

View File

@ -1,414 +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 static java.util.stream.Collectors.toList;
import static org.apache.hadoop.hbase.client.ConnectionUtils.allOf;
import static org.apache.hadoop.hbase.client.ConnectionUtils.toCheckExistenceOnly;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
/**
* The base interface for asynchronous version of Table. Obtain an instance from a
* {@link AsyncConnection}.
* <p>
* The implementation is required to be thread safe.
* <p>
* Usually the implementation will not throw any exception directly. You need to get the exception
* from the returned {@link CompletableFuture}.
* @since 2.0.0
*/
@InterfaceAudience.Public
public interface AsyncTableBase {
/**
* Gets the fully qualified table name instance of this table.
*/
TableName getName();
/**
* Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
* <p>
* The reference returned is not a copy, so any change made to it will affect this instance.
*/
Configuration getConfiguration();
/**
* Get timeout of each rpc request in this Table instance. It will be overridden by a more
* specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
* @see #getReadRpcTimeout(TimeUnit)
* @see #getWriteRpcTimeout(TimeUnit)
* @param unit the unit of time the timeout to be represented in
* @return rpc timeout in the specified time unit
*/
long getRpcTimeout(TimeUnit unit);
/**
* Get timeout of each rpc read request in this Table instance.
* @param unit the unit of time the timeout to be represented in
* @return read rpc timeout in the specified time unit
*/
long getReadRpcTimeout(TimeUnit unit);
/**
* Get timeout of each rpc write request in this Table instance.
* @param unit the unit of time the timeout to be represented in
* @return write rpc timeout in the specified time unit
*/
long getWriteRpcTimeout(TimeUnit unit);
/**
* Get timeout of each operation in Table instance.
* @param unit the unit of time the timeout to be represented in
* @return operation rpc timeout in the specified time unit
*/
long getOperationTimeout(TimeUnit unit);
/**
* Get the timeout of a single operation in a scan. It works like operation timeout for other
* operations.
* @param unit the unit of time the timeout to be represented in
* @return scan rpc timeout in the specified time unit
*/
long getScanTimeout(TimeUnit unit);
/**
* Test for the existence of columns in the table, as specified by the Get.
* <p>
* This will return true if the Get matches one or more keys, false if not.
* <p>
* This is a server-side call so it prevents any data from being transfered to the client.
* @return true if the specified Get matches one or more keys, false if not. The return value will
* be wrapped by a {@link CompletableFuture}.
*/
default CompletableFuture<Boolean> exists(Get get) {
return get(toCheckExistenceOnly(get)).thenApply(r -> r.getExists());
}
/**
* Extracts certain cells from a given row.
* @param get The object that specifies what data to fetch and from which row.
* @return The data coming from the specified row, if it exists. If the row specified doesn't
* exist, the {@link Result} instance returned won't contain any
* {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}. The
* return value will be wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Result> get(Get get);
/**
* Puts some data to the table.
* @param put The data to put.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
CompletableFuture<Void> put(Put put);
/**
* Deletes the specified cells/row.
* @param delete The object that specifies what to delete.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
CompletableFuture<Void> delete(Delete delete);
/**
* Appends values to one or more columns within a single row.
* <p>
* This operation does not appear atomic to readers. Appends are done under a single row lock, so
* write operations to a row are synchronized, but readers do not take row locks so get and scan
* operations can see this operation partially completed.
* @param append object that specifies the columns and amounts to be used for the increment
* operations
* @return values of columns after the append operation (maybe null). The return value will be
* wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Result> append(Append append);
/**
* Increments one or more columns within a single row.
* <p>
* This operation does not appear atomic to readers. Increments are done under a single row lock,
* so write operations to a row are synchronized, but readers do not take row locks so get and
* scan operations can see this operation partially completed.
* @param increment object that specifies the columns and amounts to be used for the increment
* operations
* @return values of columns after the increment. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
CompletableFuture<Result> increment(Increment increment);
/**
* See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
* <p>
* The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
* @param row The row that contains the cell to increment.
* @param family The column family of the cell to increment.
* @param qualifier The column qualifier of the cell to increment.
* @param amount The amount to increment the cell with (or decrement, if the amount is negative).
* @return The new value, post increment. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
long amount) {
return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
}
/**
* Atomically increments a column value. If the column value already exists and is not a
* big-endian long, this could throw an exception. If the column value does not yet exist it is
* initialized to <code>amount</code> and written to the specified column.
* <p>
* Setting durability to {@link Durability#SKIP_WAL} means that in a fail scenario you will lose
* any increments that have not been flushed.
* @param row The row that contains the cell to increment.
* @param family The column family of the cell to increment.
* @param qualifier The column qualifier of the cell to increment.
* @param amount The amount to increment the cell with (or decrement, if the amount is negative).
* @param durability The persistence guarantee for this increment.
* @return The new value, post increment. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
long amount, Durability durability) {
Preconditions.checkNotNull(row, "row is null");
Preconditions.checkNotNull(family, "family is null");
return increment(
new Increment(row).addColumn(family, qualifier, amount).setDurability(durability))
.thenApply(r -> Bytes.toLong(r.getValue(family, qualifier)));
}
/**
* Atomically checks if a row/family/qualifier value matches the expected value. If it does, it
* adds the Put/Delete/RowMutations.
* <p>
* Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it.
* This is a fluent style API, the code is like:
*
* <pre>
* <code>
* table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put)
* .thenAccept(succ -> {
* if (succ) {
* System.out.println("Check and put succeeded");
* } else {
* System.out.println("Check and put failed");
* }
* });
* </code>
* </pre>
*/
CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family);
/**
* A helper class for sending checkAndMutate request.
*/
interface CheckAndMutateBuilder {
/**
* @param qualifier column qualifier to check.
*/
CheckAndMutateBuilder qualifier(byte[] qualifier);
/**
* Check for lack of column.
*/
CheckAndMutateBuilder ifNotExists();
default CheckAndMutateBuilder ifEquals(byte[] value) {
return ifMatches(CompareOperator.EQUAL, value);
}
/**
* @param compareOp comparison operator to use
* @param value the expected value
*/
CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
/**
* @param put data to put if check succeeds
* @return {@code true} if the new put was executed, {@code false} otherwise. The return value
* will be wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Boolean> thenPut(Put put);
/**
* @param delete data to delete if check succeeds
* @return {@code true} if the new delete was executed, {@code false} otherwise. The return
* value will be wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Boolean> thenDelete(Delete delete);
/**
* @param mutation mutations to perform if check succeeds
* @return true if the new mutation was executed, false otherwise. The return value will be
* wrapped by a {@link CompletableFuture}.
*/
CompletableFuture<Boolean> thenMutate(RowMutations mutation);
}
/**
* Performs multiple mutations atomically on a single row. Currently {@link Put} and
* {@link Delete} are supported.
* @param mutation object that specifies the set of mutations to perform atomically
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
CompletableFuture<Void> mutateRow(RowMutations mutation);
/**
* Return all the results that match the given scan object.
* <p>
* Notice that usually you should use this method with a {@link Scan} object that has limit set.
* For example, if you want to get the closest row after a given row, you could do this:
* <p>
*
* <pre>
* <code>
* table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -> {
* if (results.isEmpty()) {
* System.out.println("No row after " + Bytes.toStringBinary(row));
* } else {
* System.out.println("The closest row after " + Bytes.toStringBinary(row) + " is "
* + Bytes.toStringBinary(results.stream().findFirst().get().getRow()));
* }
* });
* </code>
* </pre>
* <p>
* If your result set is very large, you should use other scan method to get a scanner or use
* callback to process the results. They will do chunking to prevent OOM. The scanAll method will
* fetch all the results and store them in a List and then return the list to you.
* <p>
* The scan metrics will be collected background if you enable it but you have no way to get it.
* Usually you can get scan metrics from {@code ResultScanner}, or through
* {@code ScanResultConsumer.onScanMetricsCreated} but this method only returns a list of results.
* So if you really care about scan metrics then you'd better use other scan methods which return
* a {@code ResultScanner} or let you pass in a {@code ScanResultConsumer}. There is no
* performance difference between these scan methods so do not worry.
* @param scan A configured {@link Scan} object. So if you use this method to fetch a really large
* result set, it is likely to cause OOM.
* @return The results of this small scan operation. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
CompletableFuture<List<Result>> scanAll(Scan scan);
/**
* Test for the existence of columns in the table, as specified by the Gets.
* <p>
* This will return a list of booleans. Each value will be true if the related Get matches one or
* more keys, false if not.
* <p>
* This is a server-side call so it prevents any data from being transferred to the client.
* @param gets the Gets
* @return A list of {@link CompletableFuture}s that represent the existence for each get.
*/
default List<CompletableFuture<Boolean>> exists(List<Get> gets) {
return get(toCheckExistenceOnly(gets)).stream()
.<CompletableFuture<Boolean>> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
}
/**
* A simple version for batch exists. It will fail if there are any failures and you will get the
* whole result boolean list at once if the operation is succeeded.
* @param gets the Gets
* @return A {@link CompletableFuture} that wrapper the result boolean list.
*/
default CompletableFuture<List<Boolean>> existsAll(List<Get> gets) {
return allOf(exists(gets));
}
/**
* Extracts certain cells from the given rows, in batch.
* <p>
* Notice that you may not get all the results with this function, which means some of the
* returned {@link CompletableFuture}s may succeed while some of the other returned
* {@link CompletableFuture}s may fail.
* @param gets The objects that specify what data to fetch and from which rows.
* @return A list of {@link CompletableFuture}s that represent the result for each get.
*/
List<CompletableFuture<Result>> get(List<Get> gets);
/**
* A simple version for batch get. It will fail if there are any failures and you will get the
* whole result list at once if the operation is succeeded.
* @param gets The objects that specify what data to fetch and from which rows.
* @return A {@link CompletableFuture} that wrapper the result list.
*/
default CompletableFuture<List<Result>> getAll(List<Get> gets) {
return allOf(get(gets));
}
/**
* Puts some data in the table, in batch.
* @param puts The list of mutations to apply.
* @return A list of {@link CompletableFuture}s that represent the result for each put.
*/
List<CompletableFuture<Void>> put(List<Put> puts);
/**
* A simple version of batch put. It will fail if there are any failures.
* @param puts The list of mutations to apply.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
default CompletableFuture<Void> putAll(List<Put> puts) {
return allOf(put(puts)).thenApply(r -> null);
}
/**
* Deletes the specified cells/rows in bulk.
* @param deletes list of things to delete.
* @return A list of {@link CompletableFuture}s that represent the result for each delete.
*/
List<CompletableFuture<Void>> delete(List<Delete> deletes);
/**
* A simple version of batch delete. It will fail if there are any failures.
* @param deletes list of things to delete.
* @return A {@link CompletableFuture} that always returns null when complete normally.
*/
default CompletableFuture<Void> deleteAll(List<Delete> deletes) {
return allOf(delete(deletes)).thenApply(r -> null);
}
/**
* Method that does a batch call on Deletes, Gets, Puts, Increments and Appends. The ordering of
* execution of the actions is not defined. Meaning if you do a Put and a Get in the same
* {@link #batch} call, you will not necessarily be guaranteed that the Get returns what the Put
* had put.
* @param actions list of Get, Put, Delete, Increment, Append objects
* @return A list of {@link CompletableFuture}s that represent the result for each action.
*/
<T> List<CompletableFuture<T>> batch(List<? extends Row> actions);
/**
* A simple version of batch. It will fail if there are any failures and you will get the whole
* result list at once if the operation is succeeded.
* @param actions list of Get, Put, Delete, Increment, Append objects
* @return A list of the result for the actions. Wrapped by a {@link CompletableFuture}.
*/
default <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
return allOf(batch(actions));
}
}

View File

@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
/**
* For creating {@link AsyncTable} or {@link RawAsyncTable}.
* For creating {@link AsyncTable}.
* <p>
* The implementation should have default configurations set before returning the builder to user.
* So users are free to only set the configs they care about to create a new
@ -32,7 +32,7 @@ import org.apache.yetus.audience.InterfaceAudience;
* @since 2.0.0
*/
@InterfaceAudience.Public
public interface AsyncTableBuilder<T extends AsyncTableBase> {
public interface AsyncTableBuilder<C extends ScanResultConsumerBase> {
/**
* Set timeout for a whole operation such as get, put or delete. Notice that scan will not be
@ -44,7 +44,7 @@ public interface AsyncTableBuilder<T extends AsyncTableBase> {
* @see #setMaxRetries(int)
* @see #setScanTimeout(long, TimeUnit)
*/
AsyncTableBuilder<T> setOperationTimeout(long timeout, TimeUnit unit);
AsyncTableBuilder<C> setOperationTimeout(long timeout, TimeUnit unit);
/**
* As now we have heartbeat support for scan, ideally a scan will never timeout unless the RS is
@ -53,7 +53,7 @@ public interface AsyncTableBuilder<T extends AsyncTableBase> {
* operation in a scan, such as openScanner or next.
* @see #setScanTimeout(long, TimeUnit)
*/
AsyncTableBuilder<T> setScanTimeout(long timeout, TimeUnit unit);
AsyncTableBuilder<C> setScanTimeout(long timeout, TimeUnit unit);
/**
* Set timeout for each rpc request.
@ -61,23 +61,23 @@ public interface AsyncTableBuilder<T extends AsyncTableBase> {
* Notice that this will <strong>NOT</strong> change the rpc timeout for read(get, scan) request
* and write request(put, delete).
*/
AsyncTableBuilder<T> setRpcTimeout(long timeout, TimeUnit unit);
AsyncTableBuilder<C> setRpcTimeout(long timeout, TimeUnit unit);
/**
* Set timeout for each read(get, scan) rpc request.
*/
AsyncTableBuilder<T> setReadRpcTimeout(long timeout, TimeUnit unit);
AsyncTableBuilder<C> setReadRpcTimeout(long timeout, TimeUnit unit);
/**
* Set timeout for each write(put, delete) rpc request.
*/
AsyncTableBuilder<T> setWriteRpcTimeout(long timeout, TimeUnit unit);
AsyncTableBuilder<C> setWriteRpcTimeout(long timeout, TimeUnit unit);
/**
* Set the base pause time for retrying. We use an exponential policy to generate sleep time when
* retrying.
*/
AsyncTableBuilder<T> setRetryPause(long pause, TimeUnit unit);
AsyncTableBuilder<C> setRetryPause(long pause, TimeUnit unit);
/**
* Set the max retry times for an operation. Usually it is the max attempt times minus 1.
@ -87,7 +87,7 @@ public interface AsyncTableBuilder<T extends AsyncTableBase> {
* @see #setMaxAttempts(int)
* @see #setOperationTimeout(long, TimeUnit)
*/
default AsyncTableBuilder<T> setMaxRetries(int maxRetries) {
default AsyncTableBuilder<C> setMaxRetries(int maxRetries) {
return setMaxAttempts(retries2Attempts(maxRetries));
}
@ -98,15 +98,15 @@ public interface AsyncTableBuilder<T extends AsyncTableBase> {
* @see #setMaxRetries(int)
* @see #setOperationTimeout(long, TimeUnit)
*/
AsyncTableBuilder<T> setMaxAttempts(int maxAttempts);
AsyncTableBuilder<C> setMaxAttempts(int maxAttempts);
/**
* Set the number of retries that are allowed before we start to log.
*/
AsyncTableBuilder<T> setStartLogErrorsCnt(int startLogErrorsCnt);
AsyncTableBuilder<C> setStartLogErrorsCnt(int startLogErrorsCnt);
/**
* Create the {@link AsyncTable} or {@link RawAsyncTable} instance.
* Create the {@link AsyncTable} instance.
*/
T build();
AsyncTable<C> build();
}

View File

@ -28,7 +28,8 @@ import org.apache.yetus.audience.InterfaceAudience;
* Base class for all asynchronous table builders.
*/
@InterfaceAudience.Private
abstract class AsyncTableBuilderBase<T extends AsyncTableBase> implements AsyncTableBuilder<T> {
abstract class AsyncTableBuilderBase<C extends ScanResultConsumerBase>
implements AsyncTableBuilder<C> {
protected TableName tableName;
@ -51,7 +52,7 @@ abstract class AsyncTableBuilderBase<T extends AsyncTableBase> implements AsyncT
AsyncTableBuilderBase(TableName tableName, AsyncConnectionConfiguration connConf) {
this.tableName = tableName;
this.operationTimeoutNs = tableName.isSystemTable() ? connConf.getMetaOperationTimeoutNs()
: connConf.getOperationTimeoutNs();
: connConf.getOperationTimeoutNs();
this.scanTimeoutNs = connConf.getScanTimeoutNs();
this.rpcTimeoutNs = connConf.getRpcTimeoutNs();
this.readRpcTimeoutNs = connConf.getReadRpcTimeoutNs();
@ -62,49 +63,49 @@ abstract class AsyncTableBuilderBase<T extends AsyncTableBase> implements AsyncT
}
@Override
public AsyncTableBuilderBase<T> setOperationTimeout(long timeout, TimeUnit unit) {
public AsyncTableBuilderBase<C> setOperationTimeout(long timeout, TimeUnit unit) {
this.operationTimeoutNs = unit.toNanos(timeout);
return this;
}
@Override
public AsyncTableBuilderBase<T> setScanTimeout(long timeout, TimeUnit unit) {
public AsyncTableBuilderBase<C> setScanTimeout(long timeout, TimeUnit unit) {
this.scanTimeoutNs = unit.toNanos(timeout);
return this;
}
@Override
public AsyncTableBuilderBase<T> setRpcTimeout(long timeout, TimeUnit unit) {
public AsyncTableBuilderBase<C> setRpcTimeout(long timeout, TimeUnit unit) {
this.rpcTimeoutNs = unit.toNanos(timeout);
return this;
}
@Override
public AsyncTableBuilderBase<T> setReadRpcTimeout(long timeout, TimeUnit unit) {
public AsyncTableBuilderBase<C> setReadRpcTimeout(long timeout, TimeUnit unit) {
this.readRpcTimeoutNs = unit.toNanos(timeout);
return this;
}
@Override
public AsyncTableBuilderBase<T> setWriteRpcTimeout(long timeout, TimeUnit unit) {
public AsyncTableBuilderBase<C> setWriteRpcTimeout(long timeout, TimeUnit unit) {
this.writeRpcTimeoutNs = unit.toNanos(timeout);
return this;
}
@Override
public AsyncTableBuilderBase<T> setRetryPause(long pause, TimeUnit unit) {
public AsyncTableBuilderBase<C> setRetryPause(long pause, TimeUnit unit) {
this.pauseNs = unit.toNanos(pause);
return this;
}
@Override
public AsyncTableBuilderBase<T> setMaxAttempts(int maxAttempts) {
public AsyncTableBuilderBase<C> setMaxAttempts(int maxAttempts) {
this.maxAttempts = maxAttempts;
return this;
}
@Override
public AsyncTableBuilderBase<T> setStartLogErrorsCnt(int startLogErrorsCnt) {
public AsyncTableBuilderBase<C> setStartLogErrorsCnt(int startLogErrorsCnt) {
this.startLogErrorsCnt = startLogErrorsCnt;
return this;
}

View File

@ -19,34 +19,37 @@ package org.apache.hadoop.hbase.client;
import static java.util.stream.Collectors.toList;
import com.google.protobuf.RpcChannel;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The implementation of AsyncTable. Based on {@link RawAsyncTable}.
* Just a wrapper of {@link RawAsyncTableImpl}. The difference is that users need to provide a
* thread pool when constructing this class, and the callback methods registered to the returned
* {@link CompletableFuture} will be executed in this thread pool. So usually it is safe for users
* to do anything they want in the callbacks without breaking the rpc framework.
*/
@InterfaceAudience.Private
class AsyncTableImpl implements AsyncTable {
class AsyncTableImpl implements AsyncTable<ScanResultConsumer> {
private final RawAsyncTable rawTable;
private final AsyncTable<AdvancedScanResultConsumer> rawTable;
private final ExecutorService pool;
private final long defaultScannerMaxResultSize;
AsyncTableImpl(AsyncConnectionImpl conn, RawAsyncTable rawTable, ExecutorService pool) {
AsyncTableImpl(AsyncConnectionImpl conn, AsyncTable<AdvancedScanResultConsumer> rawTable,
ExecutorService pool) {
this.rawTable = rawTable;
this.pool = pool;
this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();
}
@Override
@ -172,16 +175,9 @@ class AsyncTableImpl implements AsyncTable {
return wrap(rawTable.scanAll(scan));
}
private long resultSize2CacheSize(long maxResultSize) {
// * 2 if possible
return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;
}
@Override
public ResultScanner getScanner(Scan scan) {
return new AsyncTableResultScanner(rawTable, ReflectionUtils.newInstance(scan.getClass(), scan),
resultSize2CacheSize(
scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
return rawTable.getScanner(scan);
}
private void scan0(Scan scan, ScanResultConsumer consumer) {
@ -222,4 +218,59 @@ class AsyncTableImpl implements AsyncTable {
public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {
return rawTable.<T> batch(actions).stream().map(this::wrap).collect(toList());
}
@Override
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, byte[] row) {
return wrap(rawTable.coprocessorService(stubMaker, callable, row));
}
@Override
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
CoprocessorCallback<R> callback) {
CoprocessorCallback<R> wrappedCallback = new CoprocessorCallback<R>() {
@Override
public void onRegionComplete(RegionInfo region, R resp) {
pool.execute(() -> callback.onRegionComplete(region, resp));
}
@Override
public void onRegionError(RegionInfo region, Throwable error) {
pool.execute(() -> callback.onRegionError(region, error));
}
@Override
public void onComplete() {
pool.execute(() -> callback.onComplete());
}
@Override
public void onError(Throwable error) {
pool.execute(() -> callback.onError(error));
}
};
CoprocessorServiceBuilder<S, R> builder =
rawTable.coprocessorService(stubMaker, callable, wrappedCallback);
return new CoprocessorServiceBuilder<S, R>() {
@Override
public CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey, boolean inclusive) {
builder.fromRow(startKey, inclusive);
return this;
}
@Override
public CoprocessorServiceBuilder<S, R> toRow(byte[] endKey, boolean inclusive) {
builder.toRow(endKey, inclusive);
return this;
}
@Override
public void execute() {
builder.execute();
}
};
}
}

View File

@ -37,11 +37,11 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
* {@code 2 * scan.getMaxResultSize()}.
*/
@InterfaceAudience.Private
class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
class AsyncTableResultScanner implements ResultScanner, AdvancedScanResultConsumer {
private static final Log LOG = LogFactory.getLog(AsyncTableResultScanner.class);
private final RawAsyncTable rawTable;
private final AsyncTable<AdvancedScanResultConsumer> rawTable;
private final long maxCacheSize;
@ -59,7 +59,8 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
private ScanResumer resumer;
public AsyncTableResultScanner(RawAsyncTable table, Scan scan, long maxCacheSize) {
public AsyncTableResultScanner(AsyncTable<AdvancedScanResultConsumer> table, Scan scan,
long maxCacheSize) {
this.rawTable = table;
this.maxCacheSize = maxCacheSize;
this.scan = scan;
@ -74,8 +75,8 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
private void stopPrefetch(ScanController controller) {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("0x%x", System.identityHashCode(this)) +
" stop prefetching when scanning " + rawTable.getName() + " as the cache size " +
cacheSize + " is greater than the maxCacheSize " + maxCacheSize);
" stop prefetching when scanning " + rawTable.getName() + " as the cache size " +
cacheSize + " is greater than the maxCacheSize " + maxCacheSize);
}
resumer = controller.suspend();
}

View File

@ -21,9 +21,6 @@ import static java.util.stream.Collectors.toList;
import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.net.InetAddress;
@ -41,26 +38,28 @@ 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.HRegionInfo;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
/**
* Utility used by client connections.
@ -378,7 +377,7 @@ public final class ConnectionUtils {
}
}
static boolean noMoreResultsForScan(Scan scan, HRegionInfo info) {
static boolean noMoreResultsForScan(Scan scan, RegionInfo info) {
if (isEmptyStopRow(info.getEndKey())) {
return true;
}
@ -392,7 +391,7 @@ public final class ConnectionUtils {
return c > 0 || (c == 0 && !scan.includeStopRow());
}
static boolean noMoreResultsForReverseScan(Scan scan, HRegionInfo info) {
static boolean noMoreResultsForReverseScan(Scan scan, RegionInfo info) {
if (isEmptyStartRow(info.getStartKey())) {
return true;
}

View File

@ -19,6 +19,11 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.protobuf.Message;
import com.google.protobuf.RpcChannel;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -63,7 +68,6 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.client.replication.TableCFs;
@ -83,6 +87,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
@ -245,11 +250,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Updat
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.protobuf.Message;
import com.google.protobuf.RpcChannel;
/**
* The implementation of AsyncAdmin.
* <p>
@ -263,7 +263,7 @@ import com.google.protobuf.RpcChannel;
* @see AsyncConnection#getAdminBuilder()
*/
@InterfaceAudience.Private
public class RawAsyncHBaseAdmin implements AsyncAdmin {
class RawAsyncHBaseAdmin implements AsyncAdmin {
public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE = "flush-table-proc";
private static final Log LOG = LogFactory.getLog(AsyncHBaseAdmin.class);
@ -272,7 +272,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private final HashedWheelTimer retryTimer;
private final RawAsyncTable metaTable;
private final AsyncTable<AdvancedScanResultConsumer> metaTable;
private final long rpcTimeoutNs;
@ -290,7 +290,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
AsyncAdminBuilderBase builder) {
this.connection = connection;
this.retryTimer = retryTimer;
this.metaTable = connection.getRawTable(META_TABLE_NAME);
this.metaTable = connection.getTable(META_TABLE_NAME);
this.rpcTimeoutNs = builder.rpcTimeoutNs;
this.operationTimeoutNs = builder.operationTimeoutNs;
this.pauseNs = builder.pauseNs;
@ -1442,8 +1442,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
public CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter) {
CompletableFuture<List<QuotaSettings>> future = new CompletableFuture<>();
Scan scan = QuotaTableUtil.makeScan(filter);
this.connection.getRawTableBuilder(QuotaTableUtil.QUOTA_TABLE_NAME).build()
.scan(scan, new RawScanResultConsumer() {
this.connection.getTableBuilder(QuotaTableUtil.QUOTA_TABLE_NAME).build()
.scan(scan, new AdvancedScanResultConsumer() {
List<QuotaSettings> settings = new ArrayList<>();
@Override
@ -3001,7 +3001,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable) {
ServiceCaller<S, R> callable) {
MasterCoprocessorRpcChannelImpl channel =
new MasterCoprocessorRpcChannelImpl(this.<Message> newMasterCaller());
S stub = stubMaker.apply(channel);
@ -3019,7 +3019,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, ServerName serverName) {
ServiceCaller<S, R> callable, ServerName serverName) {
RegionServerCoprocessorRpcChannelImpl channel =
new RegionServerCoprocessorRpcChannelImpl(this.<Message> newServerCaller().serverName(
serverName));

View File

@ -1,263 +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.util.concurrent.CompletableFuture;
import java.util.function.Function;
import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.RpcController;
/**
* A low level asynchronous table.
* <p>
* The implementation is required to be thread safe.
* <p>
* The returned {@code CompletableFuture} will be finished directly in the rpc framework's callback
* thread, so typically you should not do any time consuming work inside these methods, otherwise
* you will be likely to block at least one connection to RS(even more if the rpc framework uses
* NIO).
* <p>
* So, only experts that want to build high performance service should use this interface directly,
* especially for the {@link #scan(Scan, RawScanResultConsumer)} below.
* @since 2.0.0
*/
@InterfaceAudience.Public
public interface RawAsyncTable extends AsyncTableBase {
/**
* The basic scan API uses the observer pattern. All results that match the given scan object will
* be passed to the given {@code consumer} by calling {@code RawScanResultConsumer.onNext}.
* {@code RawScanResultConsumer.onComplete} means the scan is finished, and
* {@code RawScanResultConsumer.onError} means we hit an unrecoverable error and the scan is
* terminated. {@code RawScanResultConsumer.onHeartbeat} means the RS is still working but we can
* not get a valid result to call {@code RawScanResultConsumer.onNext}. This is usually because
* the matched results are too sparse, for example, a filter which almost filters out everything
* is specified.
* <p>
* Notice that, the methods of the given {@code consumer} will be called directly in the rpc
* framework's callback thread, so typically you should not do any time consuming work inside
* these methods, otherwise you will be likely to block at least one connection to RS(even more if
* the rpc framework uses NIO).
* @param scan A configured {@link Scan} object.
* @param consumer the consumer used to receive results.
*/
void scan(Scan scan, RawScanResultConsumer consumer);
/**
* Delegate to a protobuf rpc call.
* <p>
* Usually, it is just a simple lambda expression, like:
*
* <pre>
* <code>
* (stub, controller, rpcCallback) -> {
* XXXRequest request = ...; // prepare the request
* stub.xxx(controller, request, rpcCallback);
* }
* </code>
* </pre>
*
* And if you can prepare the {@code request} before calling the coprocessorService method, the
* lambda expression will be:
*
* <pre>
* <code>
* (stub, controller, rpcCallback) -> stub.xxx(controller, request, rpcCallback)
* </code>
* </pre>
*/
@InterfaceAudience.Public
@FunctionalInterface
interface CoprocessorCallable<S, R> {
/**
* Represent the actual protobuf rpc call.
* @param stub the asynchronous stub
* @param controller the rpc controller, has already been prepared for you
* @param rpcCallback the rpc callback, has already been prepared for you
*/
void call(S stub, RpcController controller, RpcCallback<R> rpcCallback);
}
/**
* Execute the given coprocessor call on the region which contains the given {@code row}.
* <p>
* The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
* one line lambda expression, like:
*
* <pre>
* <code>
* channel -> xxxService.newStub(channel)
* </code>
* </pre>
*
* @param stubMaker a delegation to the actual {@code newStub} call.
* @param callable a delegation to the actual protobuf rpc call. See the comment of
* {@link CoprocessorCallable} for more details.
* @param row The row key used to identify the remote region location
* @param <S> the type of the asynchronous stub
* @param <R> the type of the return value
* @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
* @see CoprocessorCallable
*/
<S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, byte[] row);
/**
* The callback when we want to execute a coprocessor call on a range of regions.
* <p>
* As the locating itself also takes some time, the implementation may want to send rpc calls on
* the fly, which means we do not know how many regions we have when we get the return value of
* the rpc calls, so we need an {@link #onComplete()} which is used to tell you that we have
* passed all the return values to you(through the {@link #onRegionComplete(RegionInfo, Object)}
* or {@link #onRegionError(RegionInfo, Throwable)} calls), i.e, there will be no
* {@link #onRegionComplete(RegionInfo, Object)} or {@link #onRegionError(RegionInfo, Throwable)}
* calls in the future.
* <p>
* Here is a pseudo code to describe a typical implementation of a range coprocessor service
* method to help you better understand how the {@link CoprocessorCallback} will be called. The
* {@code callback} in the pseudo code is our {@link CoprocessorCallback}. And notice that the
* {@code whenComplete} is {@code CompletableFuture.whenComplete}.
*
* <pre>
* locateThenCall(byte[] row) {
* locate(row).whenComplete((location, locateError) -> {
* if (locateError != null) {
* callback.onError(locateError);
* return;
* }
* incPendingCall();
* region = location.getRegion();
* if (region.getEndKey() > endKey) {
* locateEnd = true;
* } else {
* locateThenCall(region.getEndKey());
* }
* sendCall().whenComplete((resp, error) -> {
* if (error != null) {
* callback.onRegionError(region, error);
* } else {
* callback.onRegionComplete(region, resp);
* }
* if (locateEnd && decPendingCallAndGet() == 0) {
* callback.onComplete();
* }
* });
* });
* }
* </pre>
*/
@InterfaceAudience.Public
interface CoprocessorCallback<R> {
/**
* @param region the region that the response belongs to
* @param resp the response of the coprocessor call
*/
void onRegionComplete(RegionInfo region, R resp);
/**
* @param region the region that the error belongs to
* @param error the response error of the coprocessor call
*/
void onRegionError(RegionInfo region, Throwable error);
/**
* Indicate that all responses of the regions have been notified by calling
* {@link #onRegionComplete(RegionInfo, Object)} or
* {@link #onRegionError(RegionInfo, Throwable)}.
*/
void onComplete();
/**
* Indicate that we got an error which does not belong to any regions. Usually a locating error.
*/
void onError(Throwable error);
}
/**
* Helper class for sending coprocessorService request that executes a coprocessor call on regions
* which are covered by a range.
* <p>
* If {@code fromRow} is not specified the selection will start with the first table region. If
* {@code toRow} is not specified the selection will continue through the last table region.
* @param <S> the type of the protobuf Service you want to call.
* @param <R> the type of the return value.
*/
interface CoprocessorServiceBuilder<S, R> {
/**
* @param startKey start region selection with region containing this row, inclusive.
*/
default CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey) {
return fromRow(startKey, true);
}
/**
* @param startKey start region selection with region containing this row
* @param inclusive whether to include the startKey
*/
CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey, boolean inclusive);
/**
* @param endKey select regions up to and including the region containing this row, exclusive.
*/
default CoprocessorServiceBuilder<S, R> toRow(byte[] endKey) {
return toRow(endKey, false);
}
/**
* @param endKey select regions up to and including the region containing this row
* @param inclusive whether to include the endKey
*/
CoprocessorServiceBuilder<S, R> toRow(byte[] endKey, boolean inclusive);
/**
* Execute the coprocessorService request. You can get the response through the
* {@link CoprocessorCallback}.
*/
void execute();
}
/**
* Execute a coprocessor call on the regions which are covered by a range.
* <p>
* Use the returned {@link CoprocessorServiceBuilder} construct your request and then execute it.
* <p>
* The {@code stubMaker} is just a delegation to the {@code xxxService.newStub} call. Usually it
* is only a one line lambda expression, like:
*
* <pre>
* <code>
* channel -> xxxService.newStub(channel)
* </code>
* </pre>
*
* @param stubMaker a delegation to the actual {@code newStub} call.
* @param callable a delegation to the actual protobuf rpc call. See the comment of
* {@link CoprocessorCallable} for more details.
* @param callback callback to get the response. See the comment of {@link CoprocessorCallback}
* for more details.
*/
<S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, CoprocessorCallback<R> callback);
}

View File

@ -62,9 +62,16 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType
/**
* The implementation of RawAsyncTable.
* <p>
* The word 'Raw' means that this is a low level class. The returned {@link CompletableFuture} will
* be finished inside the rpc framework thread, which means that the callbacks registered to the
* {@link CompletableFuture} will also be executed inside the rpc framework thread. So users who use
* this class should not try to do time consuming tasks in the callbacks.
* @since 2.0.0
* @see AsyncTableImpl
*/
@InterfaceAudience.Private
class RawAsyncTableImpl implements RawAsyncTable {
class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
private final AsyncConnectionImpl conn;
@ -102,7 +109,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
this.maxAttempts = builder.maxAttempts;
this.startLogErrorsCnt = builder.startLogErrorsCnt;
this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching()
: conn.connConf.getScannerCaching();
: conn.connConf.getScannerCaching();
this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();
}
@ -270,7 +277,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
@Override
public CheckAndMutateBuilder qualifier(byte[] qualifier) {
this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" +
" an empty byte array, or just do not call this method if you want a null qualifier");
" an empty byte array, or just do not call this method if you want a null qualifier");
return this;
}
@ -290,7 +297,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
private void preCheck() {
Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" +
" calling ifNotExists/ifEquals/ifMatches before executing the request");
" calling ifNotExists/ifEquals/ifMatches before executing the request");
}
@Override
@ -354,14 +361,12 @@ class RawAsyncTableImpl implements RawAsyncTable {
} else {
try {
org.apache.hadoop.hbase.client.MultiResponse multiResp =
ResponseConverter.getResults(req, resp, controller.cellScanner());
ResponseConverter.getResults(req, resp, controller.cellScanner());
Throwable ex = multiResp.getException(regionName);
if (ex != null) {
future
.completeExceptionally(ex instanceof IOException ? ex
: new IOException(
"Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()),
ex));
future.completeExceptionally(ex instanceof IOException ? ex
: new IOException(
"Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
} else {
future.complete(respConverter
.apply((Result) multiResp.getResults().get(regionName).result.get(0)));
@ -400,11 +405,28 @@ class RawAsyncTableImpl implements RawAsyncTable {
return newScan;
}
public void scan(Scan scan, AdvancedScanResultConsumer consumer) {
new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, pauseNs,
maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();
}
private long resultSize2CacheSize(long maxResultSize) {
// * 2 if possible
return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;
}
@Override
public ResultScanner getScanner(Scan scan) {
return new AsyncTableResultScanner(this, ReflectionUtils.newInstance(scan.getClass(), scan),
resultSize2CacheSize(
scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
}
@Override
public CompletableFuture<List<Result>> scanAll(Scan scan) {
CompletableFuture<List<Result>> future = new CompletableFuture<>();
List<Result> scanResults = new ArrayList<>();
scan(scan, new RawScanResultConsumer() {
scan(scan, new AdvancedScanResultConsumer() {
@Override
public void onNext(Result[] results, ScanController controller) {
@ -424,11 +446,6 @@ class RawAsyncTableImpl implements RawAsyncTable {
return future;
}
public void scan(Scan scan, RawScanResultConsumer consumer) {
new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, pauseNs,
maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();
}
@Override
public List<CompletableFuture<Result>> get(List<Get> gets) {
return batch(gets, readRpcTimeoutNs);
@ -487,7 +504,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
}
private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, RegionInfo region, byte[] row) {
ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {
RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
region, row, rpcTimeoutNs, operationTimeoutNs);
S stub = stubMaker.apply(channel);
@ -505,7 +522,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
@Override
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, byte[] row) {
ServiceCaller<S, R> callable, byte[] row) {
return coprocessorService(stubMaker, callable, null, row);
}
@ -527,7 +544,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
}
private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, CoprocessorCallback<R> callback,
ServiceCaller<S, R> callable, CoprocessorCallback<R> callback,
List<HRegionLocation> locs, byte[] endKey, boolean endKeyInclusive,
AtomicBoolean locateFinished, AtomicInteger unfinishedRequest, HRegionLocation loc,
Throwable error) {
@ -563,7 +580,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
private final Function<RpcChannel, S> stubMaker;
private final CoprocessorCallable<S, R> callable;
private final ServiceCaller<S, R> callable;
private final CoprocessorCallback<R> callback;
@ -576,7 +593,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
private boolean endKeyInclusive;
public CoprocessorServiceBuilderImpl(Function<RpcChannel, S> stubMaker,
CoprocessorCallable<S, R> callable, CoprocessorCallback<R> callback) {
ServiceCaller<S, R> callable, CoprocessorCallback<R> callback) {
this.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null");
this.callable = Preconditions.checkNotNull(callable, "callable is null");
this.callback = Preconditions.checkNotNull(callback, "callback is null");
@ -586,8 +603,8 @@ class RawAsyncTableImpl implements RawAsyncTable {
public CoprocessorServiceBuilderImpl<S, R> fromRow(byte[] startKey, boolean inclusive) {
this.startKey = Preconditions.checkNotNull(startKey,
"startKey is null. Consider using" +
" an empty byte array, or just do not call this method if you want to start selection" +
" from the first region");
" an empty byte array, or just do not call this method if you want to start selection" +
" from the first region");
this.startKeyInclusive = inclusive;
return this;
}
@ -596,8 +613,8 @@ class RawAsyncTableImpl implements RawAsyncTable {
public CoprocessorServiceBuilderImpl<S, R> toRow(byte[] endKey, boolean inclusive) {
this.endKey = Preconditions.checkNotNull(endKey,
"endKey is null. Consider using" +
" an empty byte array, or just do not call this method if you want to continue" +
" selection to the last region");
" an empty byte array, or just do not call this method if you want to continue" +
" selection to the last region");
this.endKeyInclusive = inclusive;
return this;
}
@ -614,7 +631,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
@Override
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
Function<RpcChannel, S> stubMaker, CoprocessorCallable<S, R> callable,
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
CoprocessorCallback<R> callback) {
return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
}

View File

@ -18,38 +18,20 @@
package org.apache.hadoop.hbase.client;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
/**
* Receives {@link Result} for an asynchronous scan.
* <p>
* All results that match the given scan object will be passed to this class by calling
* {@link #onNext(Result)}. {@link #onComplete()} means the scan is finished, and
* {@link #onError(Throwable)} means we hit an unrecoverable error and the scan is terminated.
*/
@InterfaceAudience.Public
public interface ScanResultConsumer {
public interface ScanResultConsumer extends ScanResultConsumerBase {
/**
* @param result the data fetched from HBase service.
* @return {@code false} if you want to terminate the scan process. Otherwise {@code true}
*/
boolean onNext(Result result);
/**
* Indicate that we hit an unrecoverable error and the scan operation is terminated.
* <p>
* We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
*/
void onError(Throwable error);
/**
* Indicate that the scan operation is completed normally.
*/
void onComplete();
/**
* If {@code scan.isScanMetricsEnabled()} returns true, then this method will be called prior to
* all other methods in this interface to give you the {@link ScanMetrics} instance for this scan
* operation. The {@link ScanMetrics} instance will be updated on-the-fly during the scan, you can
* store it somewhere to get the metrics at any time if you want.
*/
default void onScanMetricsCreated(ScanMetrics scanMetrics) {
}
}

View File

@ -0,0 +1,48 @@
/**
* 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 org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The base interface for scan result consumer.
*/
@InterfaceAudience.Public
public interface ScanResultConsumerBase {
/**
* Indicate that we hit an unrecoverable error and the scan operation is terminated.
* <p>
* We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
*/
void onError(Throwable error);
/**
* Indicate that the scan operation is completed normally.
*/
void onComplete();
/**
* If {@code scan.isScanMetricsEnabled()} returns true, then this method will be called prior to
* all other methods in this interface to give you the {@link ScanMetrics} instance for this scan
* operation. The {@link ScanMetrics} instance will be updated on-the-fly during the scan, you can
* store it somewhere to get the metrics at any time if you want.
*/
default void onScanMetricsCreated(ScanMetrics scanMetrics) {
}
}

View File

@ -0,0 +1,61 @@
/**
* 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.RpcCallback;
import com.google.protobuf.RpcController;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Delegate to a protobuf rpc call.
* <p>
* Usually, it is just a simple lambda expression, like:
*
* <pre>
* <code>
* (stub, controller, rpcCallback) -> {
* XXXRequest request = ...; // prepare the request
* stub.xxx(controller, request, rpcCallback);
* }
* </code>
* </pre>
*
* And if already have the {@code request}, the lambda expression will be:
*
* <pre>
* <code>
* (stub, controller, rpcCallback) -> stub.xxx(controller, request, rpcCallback)
* </code>
* </pre>
*
* @param <S> the type of the protobuf Service you want to call.
* @param <R> the type of the return value.
*/
@InterfaceAudience.Public
@FunctionalInterface
public interface ServiceCaller<S, R> {
/**
* Represent the actual protobuf rpc call.
* @param stub the asynchronous stub
* @param controller the rpc controller, has already been prepared for you
* @param rpcCallback the rpc callback, has already been prepared for you
*/
void call(S stub, RpcController controller, RpcCallback<R> rpcCallback);
}

View File

@ -32,9 +32,9 @@ import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.RawAsyncTable;
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallback;
import org.apache.hadoop.hbase.client.RawScanResultConsumer;
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.AsyncTable.CoprocessorCallback;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@ -126,7 +126,7 @@ public class AsyncAggregationClient {
}
public static <R, S, P extends Message, Q extends Message, T extends Message> CompletableFuture<R>
max(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
max(AsyncTable<?> table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<R> future = new CompletableFuture<>();
AggregateRequest req;
try {
@ -163,7 +163,7 @@ public class AsyncAggregationClient {
}
public static <R, S, P extends Message, Q extends Message, T extends Message> CompletableFuture<R>
min(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
min(AsyncTable<?> table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<R> future = new CompletableFuture<>();
AggregateRequest req;
try {
@ -201,7 +201,7 @@ public class AsyncAggregationClient {
public static <R, S, P extends Message, Q extends Message, T extends Message>
CompletableFuture<Long>
rowCount(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
rowCount(AsyncTable<?> table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<Long> future = new CompletableFuture<>();
AggregateRequest req;
try {
@ -233,7 +233,7 @@ public class AsyncAggregationClient {
}
public static <R, S, P extends Message, Q extends Message, T extends Message> CompletableFuture<S>
sum(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
sum(AsyncTable<?> table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<S> future = new CompletableFuture<>();
AggregateRequest req;
try {
@ -269,7 +269,7 @@ public class AsyncAggregationClient {
public static <R, S, P extends Message, Q extends Message, T extends Message>
CompletableFuture<Double>
avg(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
avg(AsyncTable<?> table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<Double> future = new CompletableFuture<>();
AggregateRequest req;
try {
@ -307,7 +307,7 @@ public class AsyncAggregationClient {
public static <R, S, P extends Message, Q extends Message, T extends Message>
CompletableFuture<Double>
std(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
std(AsyncTable<?> table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<Double> future = new CompletableFuture<>();
AggregateRequest req;
try {
@ -351,7 +351,7 @@ public class AsyncAggregationClient {
// the map key is the startRow of the region
private static <R, S, P extends Message, Q extends Message, T extends Message>
CompletableFuture<NavigableMap<byte[], S>>
sumByRegion(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
sumByRegion(AsyncTable<?> table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<NavigableMap<byte[], S>> future =
new CompletableFuture<NavigableMap<byte[], S>>();
AggregateRequest req;
@ -388,8 +388,8 @@ public class AsyncAggregationClient {
}
private static <R, S, P extends Message, Q extends Message, T extends Message> void findMedian(
CompletableFuture<R> future, RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci,
Scan scan, NavigableMap<byte[], S> sumByRegion) {
CompletableFuture<R> future, AsyncTable<AdvancedScanResultConsumer> table,
ColumnInterpreter<R, S, P, Q, T> ci, Scan scan, NavigableMap<byte[], S> sumByRegion) {
double halfSum = ci.divideForAvg(sumByRegion.values().stream().reduce(ci::add).get(), 2L);
S movingSum = null;
byte[] startRow = null;
@ -410,7 +410,7 @@ public class AsyncAggregationClient {
NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(family);
byte[] weightQualifier = qualifiers.last();
byte[] valueQualifier = qualifiers.first();
table.scan(scan, new RawScanResultConsumer() {
table.scan(scan, new AdvancedScanResultConsumer() {
private S sum = baseSum;
@ -456,8 +456,9 @@ public class AsyncAggregationClient {
});
}
public static <R, S, P extends Message, Q extends Message, T extends Message> CompletableFuture<R>
median(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
public static <R, S, P extends Message, Q extends Message, T extends Message>
CompletableFuture<R> median(AsyncTable<AdvancedScanResultConsumer> table,
ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
CompletableFuture<R> future = new CompletableFuture<>();
sumByRegion(table, ci, scan).whenComplete((sumByRegion, error) -> {
if (error != null) {

View File

@ -55,7 +55,7 @@ public class TestAsyncAggregationClient {
private static AsyncConnection CONN;
private static RawAsyncTable TABLE;
private static AsyncTable<AdvancedScanResultConsumer> TABLE;
@BeforeClass
public static void setUp() throws Exception {
@ -69,7 +69,7 @@ public class TestAsyncAggregationClient {
}
UTIL.createTable(TABLE_NAME, CF, splitKeys);
CONN = ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get();
TABLE = CONN.getRawTable(TABLE_NAME);
TABLE = CONN.getTable(TABLE_NAME);
TABLE.putAll(LongStream.range(0, COUNT)
.mapToObj(l -> new Put(Bytes.toBytes(String.format("%03d", l)))
.addColumn(CF, CQ, Bytes.toBytes(l)).addColumn(CF, CQ2, Bytes.toBytes(l * l)))

View File

@ -141,7 +141,7 @@ public class AsyncClientExample extends Configured implements Tool {
latch.countDown();
return;
}
AsyncTable table = conn.getTable(tableName, threadPool);
AsyncTable<?> table = conn.getTable(tableName, threadPool);
table.put(new Put(getKey(i)).addColumn(FAMILY, QUAL, Bytes.toBytes(i)))
.whenComplete((putResp, putErr) -> {
if (putErr != null) {

View File

@ -17,9 +17,24 @@
*/
package org.apache.hadoop.hbase.client.example;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.charset.StandardCharsets;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
import org.apache.hadoop.hbase.shaded.io.netty.bootstrap.ServerBootstrap;
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
@ -43,26 +58,10 @@ import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.HttpVersion;
import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.http.QueryStringDecoder;
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.GlobalEventExecutor;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.charset.StandardCharsets;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RawAsyncTable;
import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A simple example on how to use {@link RawAsyncTable} to write a fully asynchronous HTTP proxy
* server. The {@link AsyncConnection} will share the same event loop with the HTTP server.
* A simple example on how to use {@link org.apache.hadoop.hbase.client.AsyncTable} to write a fully
* asynchronous HTTP proxy server. The {@link AsyncConnection} will share the same event loop with
* the HTTP server.
* <p>
* The request URL is:
*
@ -160,7 +159,7 @@ public class HttpProxyExample {
private void get(ChannelHandlerContext ctx, FullHttpRequest req) {
Params params = parse(req);
conn.getRawTable(TableName.valueOf(params.table)).get(new Get(Bytes.toBytes(params.row))
conn.getTable(TableName.valueOf(params.table)).get(new Get(Bytes.toBytes(params.row))
.addColumn(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier)))
.whenComplete((r, e) -> {
if (e != null) {
@ -181,7 +180,7 @@ public class HttpProxyExample {
Params params = parse(req);
byte[] value = new byte[req.content().readableBytes()];
req.content().readBytes(value);
conn.getRawTable(TableName.valueOf(params.table)).put(new Put(Bytes.toBytes(params.row))
conn.getTable(TableName.valueOf(params.table)).put(new Put(Bytes.toBytes(params.row))
.addColumn(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier), value))
.whenComplete((r, e) -> {
if (e != null) {

View File

@ -18,6 +18,11 @@
*/
package org.apache.hadoop.hbase;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.UniformReservoir;
import com.fasterxml.jackson.databind.MapperFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
import java.io.PrintStream;
import java.lang.reflect.Constructor;
@ -61,7 +66,6 @@ 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.RawAsyncTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RowMutations;
@ -80,8 +84,6 @@ import org.apache.hadoop.hbase.io.hfile.RandomDistribution;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects;
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration;
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.trace.TraceUtil;
@ -105,10 +107,8 @@ import org.apache.htrace.core.Sampler;
import org.apache.htrace.core.TraceScope;
import org.apache.yetus.audience.InterfaceAudience;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.UniformReservoir;
import com.fasterxml.jackson.databind.MapperFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects;
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* Script used evaluating HBase performance and scalability. Runs a HBase
@ -1302,7 +1302,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
}
static abstract class AsyncTableTest extends AsyncTest {
protected RawAsyncTable table;
protected AsyncTable<?> table;
AsyncTableTest(AsyncConnection con, TestOptions options, Status status) {
super(con, options, status);
@ -1310,7 +1310,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
@Override
void onStartup() throws IOException {
this.table = connection.getRawTable(TableName.valueOf(opts.tableName));
this.table = connection.getTable(TableName.valueOf(opts.tableName));
}
@Override
@ -1435,7 +1435,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
static class AsyncScanTest extends AsyncTableTest {
private ResultScanner testScanner;
private AsyncTable asyncTable;
private AsyncTable<?> asyncTable;
AsyncScanTest(AsyncConnection con, TestOptions options, Status status) {
super(con, options, status);

View File

@ -24,6 +24,7 @@ import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.ForkJoinPool;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@ -63,7 +64,7 @@ public abstract class AbstractTestAsyncTableScan {
TEST_UTIL.createTable(TABLE_NAME, FAMILY, splitKeys);
TEST_UTIL.waitTableAvailable(TABLE_NAME);
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
ASYNC_CONN.getRawTable(TABLE_NAME).putAll(IntStream.range(0, COUNT)
ASYNC_CONN.getTable(TABLE_NAME).putAll(IntStream.range(0, COUNT)
.mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
.addColumn(FAMILY, CQ1, Bytes.toBytes(i)).addColumn(FAMILY, CQ2, Bytes.toBytes(i * i)))
.collect(Collectors.toList())).get();
@ -92,7 +93,15 @@ public abstract class AbstractTestAsyncTableScan {
return new Scan().setBatch(1).setMaxResultSize(1);
}
protected static List<Pair<String, Supplier<Scan>>> getScanCreater() {
protected static AsyncTable<?> getRawTable() {
return ASYNC_CONN.getTable(TABLE_NAME);
}
protected static AsyncTable<?> getTable() {
return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
}
private static List<Pair<String, Supplier<Scan>>> getScanCreator() {
return Arrays.asList(Pair.newPair("normal", AbstractTestAsyncTableScan::createNormalScan),
Pair.newPair("batch", AbstractTestAsyncTableScan::createBatchScan),
Pair.newPair("smallResultSize", AbstractTestAsyncTableScan::createSmallResultSizeScan),
@ -100,6 +109,25 @@ public abstract class AbstractTestAsyncTableScan {
AbstractTestAsyncTableScan::createBatchSmallResultSizeScan));
}
protected static List<Object[]> getScanCreatorParams() {
return getScanCreator().stream().map(p -> new Object[] { p.getFirst(), p.getSecond() })
.collect(Collectors.toList());
}
private static List<Pair<String, Supplier<AsyncTable<?>>>> getTableCreator() {
return Arrays.asList(Pair.newPair("raw", AbstractTestAsyncTableScan::getRawTable),
Pair.newPair("normal", AbstractTestAsyncTableScan::getTable));
}
protected static List<Object[]> getTableAndScanCreatorParams() {
List<Pair<String, Supplier<AsyncTable<?>>>> tableCreator = getTableCreator();
List<Pair<String, Supplier<Scan>>> scanCreator = getScanCreator();
return tableCreator.stream()
.flatMap(tp -> scanCreator.stream().map(
sp -> new Object[] { tp.getFirst(), tp.getSecond(), sp.getFirst(), sp.getSecond() }))
.collect(Collectors.toList());
}
protected abstract Scan createScan();
protected abstract List<Result> doScan(Scan scan) throws Exception;
@ -121,10 +149,11 @@ public abstract class AbstractTestAsyncTableScan {
List<Result> results = doScan(createScan());
// make sure all scanners are closed at RS side
TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
.forEach(rs -> assertEquals(
"The scanner count of " + rs.getServerName() + " is " +
.forEach(
rs -> assertEquals(
"The scanner count of " + rs.getServerName() + " is " +
rs.getRSRpcServices().getScannersCount(),
0, rs.getRSRpcServices().getScannersCount()));
0, rs.getRSRpcServices().getScannersCount()));
assertEquals(COUNT, results.size());
IntStream.range(0, COUNT).forEach(i -> {
Result result = results.get(i);
@ -150,7 +179,7 @@ public abstract class AbstractTestAsyncTableScan {
public void testScanNoStopKey() throws Exception {
int start = 345;
List<Result> results =
doScan(createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))));
doScan(createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))));
assertEquals(COUNT - start, results.size());
IntStream.range(0, COUNT - start).forEach(i -> assertResultEquals(results.get(i), start + i));
}
@ -169,16 +198,16 @@ public abstract class AbstractTestAsyncTableScan {
try {
doScan(createScan().addFamily(Bytes.toBytes("WrongColumnFamily")));
} catch (Exception e) {
assertTrue(e instanceof NoSuchColumnFamilyException
|| e.getCause() instanceof NoSuchColumnFamilyException);
assertTrue(e instanceof NoSuchColumnFamilyException ||
e.getCause() instanceof NoSuchColumnFamilyException);
}
}
private void testScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
int limit) throws Exception {
Scan scan =
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive);
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive);
if (limit > 0) {
scan.setLimit(limit);
}
@ -195,9 +224,9 @@ public abstract class AbstractTestAsyncTableScan {
private void testReversedScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
int limit) throws Exception {
Scan scan = createScan()
.withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive).setReversed(true);
Scan scan =
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive).setReversed(true);
if (limit > 0) {
scan.setLimit(limit);
}

View File

@ -25,7 +25,12 @@ import java.util.Queue;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
class SimpleRawScanResultConsumer implements RawScanResultConsumer {
/**
* A scan result consumer which buffers all the data in memory and you can call the {@link #take()}
* method below to get the result one by one. Should only be used by tests, do not write production
* code like this as the buffer is unlimited and may cause OOM.
*/
class BufferingScanResultConsumer implements AdvancedScanResultConsumer {
private ScanMetrics scanMetrics;

View File

@ -96,7 +96,7 @@ public class TestAsyncBufferMutator {
}
// mutator.close will call mutator.flush automatically so all tasks should have been done.
futures.forEach(f -> f.join());
RawAsyncTable table = CONN.getRawTable(TABLE_NAME);
AsyncTable<?> table = CONN.getTable(TABLE_NAME);
IntStream.range(0, COUNT).mapToObj(i -> new Get(Bytes.toBytes(i))).map(g -> table.get(g).join())
.forEach(r -> {
assertArrayEquals(VALUE, r.getValue(CF, CQ));

View File

@ -212,7 +212,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws Exception {
createTableWithDefaultConf(tableName);
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
@ -305,7 +305,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(table);
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
admin.createTable(builder.build(), Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), 16).join();
RawAsyncTable asyncTable = ASYNC_CONN.getRawTable(table);
AsyncTable<?> asyncTable = ASYNC_CONN.getTable(table);
List<Put> puts = new ArrayList<>();
for (byte[] row : HBaseTestingUtility.ROWS) {
puts.add(new Put(row).addColumn(FAMILY, Bytes.toBytes("q"), Bytes.toBytes("v")));

View File

@ -69,12 +69,12 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(tableName);
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm")).get();
RegionInfo region = regionLocation.getRegionInfo();
byte[] regionName = regionLocation.getRegionInfo().getRegionName();
RegionInfo region = regionLocation.getRegion();
byte[] regionName = regionLocation.getRegion().getRegionName();
HRegionLocation location = rawAdmin.getRegionLocation(regionName).get();
assertTrue(Bytes.equals(regionName, location.getRegionInfo().getRegionName()));
assertTrue(Bytes.equals(regionName, location.getRegion().getRegionName()));
location = rawAdmin.getRegionLocation(region.getEncodedNameAsBytes()).get();
assertTrue(Bytes.equals(regionName, location.getRegionInfo().getRegionName()));
assertTrue(Bytes.equals(regionName, location.getRegion().getRegionName()));
}
@Test
@ -252,7 +252,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
.filter(rs -> rs.getServerName().equals(serverName)).findFirst().get();
// write a put into the specific region
ASYNC_CONN.getRawTable(tableName)
ASYNC_CONN.getTable(tableName)
.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-1")))
.join();
Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0);
@ -268,7 +268,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize(), 0);
// write another put into the specific region
ASYNC_CONN.getRawTable(tableName)
ASYNC_CONN.getTable(tableName)
.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-2")))
.join();
Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0);
@ -288,7 +288,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
final int rows = 10000;
loadData(tableName, families, rows);
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
List<HRegionLocation> regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
int originalCount = regionLocations.size();
@ -319,7 +319,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
byte[][] families = { FAMILY };
loadData(tableName, families, 1000);
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
List<HRegionLocation> regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
int originalCount = regionLocations.size();
@ -364,7 +364,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
createTableWithDefaultConf(tableName, splitRows);
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
List<HRegionLocation> regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
RegionInfo regionA;
@ -372,16 +372,16 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
// merge with full name
assertEquals(3, regionLocations.size());
regionA = regionLocations.get(0).getRegionInfo();
regionB = regionLocations.get(1).getRegionInfo();
regionA = regionLocations.get(0).getRegion();
regionB = regionLocations.get(1).getRegion();
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
assertEquals(2, regionLocations.size());
// merge with encoded name
regionA = regionLocations.get(0).getRegionInfo();
regionB = regionLocations.get(1).getRegionInfo();
regionA = regionLocations.get(0).getRegion();
regionB = regionLocations.get(1).getRegion();
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
regionLocations =
@ -404,12 +404,12 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
// create table
createTableWithDefaultConf(tableName);
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
List<HRegionLocation> regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
assertEquals(1, regionLocations.size());
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
List<Put> puts = new ArrayList<>();
for (int i = 0; i < rowCount; i++) {
Put put = new Put(Bytes.toBytes(i));
@ -420,9 +420,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
if (isSplitRegion) {
if (splitPoint == null) {
admin.splitRegion(regionLocations.get(0).getRegionInfo().getRegionName()).get();
admin.splitRegion(regionLocations.get(0).getRegion().getRegionName()).get();
} else {
admin.splitRegion(regionLocations.get(0).getRegionInfo().getRegionName(), splitPoint).get();
admin.splitRegion(regionLocations.get(0).getRegion().getRegionName(), splitPoint).get();
}
} else {
if (splitPoint == null) {
@ -585,7 +585,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
private static void loadData(final TableName tableName, final byte[][] families, final int rows,
final int flushes) throws IOException {
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
List<Put> puts = new ArrayList<>(rows);
byte[] qualifier = Bytes.toBytes("val");
for (int i = 0; i < flushes; i++) {

View File

@ -63,7 +63,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(2);
TEST_UTIL.getAdmin().setBalancerRunning(false, true);
TEST_UTIL.getAdmin().balancerSwitch(false, true);
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
TEST_UTIL.waitTableAvailable(TABLE_NAME);
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
@ -81,15 +81,15 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
public void testRegionMove() throws InterruptedException, ExecutionException, IOException {
// This will leave a cached entry in location cache
HRegionLocation loc = CONN.getRegionLocator(TABLE_NAME).getRegionLocation(ROW).get();
int index = TEST_UTIL.getHBaseCluster().getServerWith(loc.getRegionInfo().getRegionName());
TEST_UTIL.getAdmin().move(loc.getRegionInfo().getEncodedNameAsBytes(), Bytes.toBytes(
int index = TEST_UTIL.getHBaseCluster().getServerWith(loc.getRegion().getRegionName());
TEST_UTIL.getAdmin().move(loc.getRegion().getEncodedNameAsBytes(), Bytes.toBytes(
TEST_UTIL.getHBaseCluster().getRegionServer(1 - index).getServerName().getServerName()));
RawAsyncTable table = CONN.getRawTableBuilder(TABLE_NAME)
AsyncTable<?> table = CONN.getTableBuilder(TABLE_NAME)
.setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(30).build();
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
// move back
TEST_UTIL.getAdmin().move(loc.getRegionInfo().getEncodedNameAsBytes(),
TEST_UTIL.getAdmin().move(loc.getRegion().getEncodedNameAsBytes(),
Bytes.toBytes(loc.getServerName().getServerName()));
Result result = table.get(new Get(ROW).addColumn(FAMILY, QUALIFIER)).get();
assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER));
@ -166,7 +166,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
return mockedLocator;
}
}) {
RawAsyncTable table = mockedConn.getRawTableBuilder(TABLE_NAME)
AsyncTable<?> table = mockedConn.getTableBuilder(TABLE_NAME)
.setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(5).build();
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
assertTrue(errorTriggered.get());

View File

@ -78,13 +78,13 @@ public class TestAsyncTable {
private byte[] row;
@Parameter
public Supplier<AsyncTableBase> getTable;
public Supplier<AsyncTable<?>> getTable;
private static RawAsyncTable getRawTable() {
return ASYNC_CONN.getRawTable(TABLE_NAME);
private static AsyncTable<?> getRawTable() {
return ASYNC_CONN.getTable(TABLE_NAME);
}
private static AsyncTable getTable() {
private static AsyncTable<?> getTable() {
return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
}
@ -115,7 +115,7 @@ public class TestAsyncTable {
@Test
public void testSimple() throws Exception {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
assertTrue(table.exists(new Get(row).addColumn(FAMILY, QUALIFIER)).get());
Result result = table.get(new Get(row).addColumn(FAMILY, QUALIFIER)).get();
@ -132,7 +132,7 @@ public class TestAsyncTable {
@Test
public void testSimpleMultiple() throws Exception {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
int count = 100;
CountDownLatch putLatch = new CountDownLatch(count);
IntStream.range(0, count).forEach(
@ -176,7 +176,7 @@ public class TestAsyncTable {
@Test
public void testIncrement() throws InterruptedException, ExecutionException {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
int count = 100;
CountDownLatch latch = new CountDownLatch(count);
AtomicLong sum = new AtomicLong(0L);
@ -193,7 +193,7 @@ public class TestAsyncTable {
@Test
public void testAppend() throws InterruptedException, ExecutionException {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
int count = 10;
CountDownLatch latch = new CountDownLatch(count);
char suffix = ':';
@ -216,7 +216,7 @@ public class TestAsyncTable {
@Test
public void testCheckAndPut() throws InterruptedException, ExecutionException {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
AtomicInteger successCount = new AtomicInteger(0);
AtomicInteger successIndex = new AtomicInteger(-1);
int count = 10;
@ -238,7 +238,7 @@ public class TestAsyncTable {
@Test
public void testCheckAndDelete() throws InterruptedException, ExecutionException {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
int count = 10;
CountDownLatch putLatch = new CountDownLatch(count + 1);
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).thenRun(() -> putLatch.countDown());
@ -275,7 +275,7 @@ public class TestAsyncTable {
@Test
public void testMutateRow() throws InterruptedException, ExecutionException, IOException {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
RowMutations mutation = new RowMutations(row);
mutation.add(new Put(row).addColumn(FAMILY, concat(QUALIFIER, 1), VALUE));
table.mutateRow(mutation).get();
@ -293,7 +293,7 @@ public class TestAsyncTable {
@Test
public void testCheckAndMutate() throws InterruptedException, ExecutionException {
AsyncTableBase table = getTable.get();
AsyncTable<?> table = getTable.get();
int count = 10;
CountDownLatch putLatch = new CountDownLatch(count + 1);
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).thenRun(() -> putLatch.countDown());

View File

@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -39,7 +38,6 @@ import java.util.regex.Pattern;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -61,7 +59,7 @@ import org.junit.runners.Parameterized;
* Class to test asynchronous table admin operations.
*/
@RunWith(Parameterized.class)
@Category({LargeTests.class, ClientTests.class})
@Category({ LargeTests.class, ClientTests.class })
public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
@Test
@ -153,25 +151,25 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
}
private TableState.State getStateFromMeta(TableName table) throws Exception {
Optional<TableState> state = AsyncMetaTableAccessor.getTableState(
ASYNC_CONN.getRawTable(TableName.META_TABLE_NAME), table).get();
Optional<TableState> state = AsyncMetaTableAccessor
.getTableState(ASYNC_CONN.getTable(TableName.META_TABLE_NAME), table).get();
assertTrue(state.isPresent());
return state.get().getState();
}
@Test
public void testCreateTableNumberOfRegions() throws Exception {
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
createTableWithDefaultConf(tableName);
List<HRegionLocation> regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
assertEquals("Table should have only 1 region", 1, regionLocations.size());
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "_2");
createTableWithDefaultConf(tableName2, new byte[][] { new byte[] { 42 } });
regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
assertEquals("Table should have only 2 region", 2, regionLocations.size());
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "_3");
@ -179,7 +177,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
admin.createTable(builder.build(), "a".getBytes(), "z".getBytes(), 3).join();
regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
assertEquals("Table should have only 3 region", 3, regionLocations.size());
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
@ -197,15 +195,15 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
admin.createTable(builder.build(), new byte[] { 1 }, new byte[] { 127 }, 16).join();
regionLocations =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName5)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName5)).get();
assertEquals("Table should have 16 region", 16, regionLocations.size());
}
@Test
public void testCreateTableWithRegions() throws Exception {
byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 }, };
new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 }, };
int expectedRegions = splitKeys.length + 1;
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
createTableWithDefaultConf(tableName, splitKeys);
@ -213,9 +211,9 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
List<HRegionLocation> regions =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
Iterator<HRegionLocation> hris = regions.iterator();
assertEquals(
@ -223,36 +221,36 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
HRegionInfo hri;
RegionInfo hri;
hris = regions.iterator();
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
if (tablesOnMaster) {
@ -274,41 +272,41 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
regions =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
assertEquals(
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
hris = regions.iterator();
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
hri = hris.next().getRegionInfo();
hri = hris.next().getRegion();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
if (tablesOnMaster) {
@ -327,7 +325,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
regions =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
assertEquals(
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
expectedRegions, regions.size());
@ -339,8 +337,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
// Try an invalid case where there are duplicate split keys
splitKeys = new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 },
new byte[] { 3, 3, 3 }, new byte[] { 2, 2, 2 } };
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");;
new byte[] { 3, 3, 3 }, new byte[] { 2, 2, 2 } };
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
try {
createTableWithDefaultConf(tableName4, splitKeys);
fail("Should not be able to create this table because of " + "duplicate split keys");
@ -353,10 +351,10 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
throws IOException {
int numRS = ((ClusterConnection) TEST_UTIL.getConnection()).getCurrentNrHRS();
Map<ServerName, List<HRegionInfo>> server2Regions = new HashMap<>();
Map<ServerName, List<RegionInfo>> server2Regions = new HashMap<>();
regions.stream().forEach((loc) -> {
ServerName server = loc.getServerName();
server2Regions.computeIfAbsent(server, (s) -> new ArrayList<>()).add(loc.getRegionInfo());
server2Regions.computeIfAbsent(server, (s) -> new ArrayList<>()).add(loc.getRegion());
});
if (numRS >= 2) {
// Ignore the master region server,
@ -424,7 +422,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
// Create & Fill the table
createTableWithDefaultConf(tableName, splitKeys);
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
int expectedRows = 10;
for (int i = 0; i < expectedRows; i++) {
byte[] data = Bytes.toBytes(String.valueOf(i));
@ -449,7 +447,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
@Test
public void testDisableAndEnableTable() throws Exception {
createTableWithDefaultConf(tableName);
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
final byte[] row = Bytes.toBytes("row");
final byte[] qualifier = Bytes.toBytes("qualifier");
final byte[] value = Bytes.toBytes("value");
@ -502,8 +500,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "2");
createTableWithDefaultConf(tableName1);
createTableWithDefaultConf(tableName2);
RawAsyncTable table1 = ASYNC_CONN.getRawTable(tableName1);
RawAsyncTable table2 = ASYNC_CONN.getRawTable(tableName1);
AsyncTable<?> table1 = ASYNC_CONN.getTable(tableName1);
AsyncTable<?> table2 = ASYNC_CONN.getTable(tableName1);
final byte[] row = Bytes.toBytes("row");
final byte[] qualifier = Bytes.toBytes("qualifier");
@ -517,8 +515,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
table1.get(get).get();
table2.get(get).get();
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false)
.get().forEach(t -> admin.disableTable(t).join());
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false).get()
.forEach(t -> admin.disableTable(t).join());
// Test that tables are disabled
get = new Get(row);
@ -541,8 +539,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName1));
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName2));
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false)
.get().forEach(t -> admin.enableTable(t).join());
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false).get()
.forEach(t -> admin.enableTable(t).join());
// Test that tables are enabled
try {
@ -562,16 +560,15 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
@Test
public void testEnableTableRetainAssignment() throws Exception {
byte[][] splitKeys =
{ new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 } };
byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 } };
int expectedRegions = splitKeys.length + 1;
createTableWithDefaultConf(tableName, splitKeys);
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
List<HRegionLocation> regions =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
assertEquals(
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
expectedRegions, regions.size());
@ -582,7 +579,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
admin.enableTable(tableName).join();
List<HRegionLocation> regions2 =
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
// Check the assignment.
assertEquals(regions.size(), regions2.size());
assertTrue(regions2.containsAll(regions));
@ -611,8 +608,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
verifyTableDescriptor(tableName, FAMILY_0);
// Modify the table removing one family and verify the descriptor
admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.of(FAMILY_1))
.join();
admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.of(FAMILY_1)).join();
verifyTableDescriptor(tableName, FAMILY_0, FAMILY_1);
}
@ -632,8 +628,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
try {
// Add same column family again - expect failure
this.admin.addColumnFamily(tableName,
ColumnFamilyDescriptorBuilder.of(FAMILY_1)).join();
this.admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.of(FAMILY_1)).join();
Assert.fail("Delete a non-exist column family should fail");
} catch (Exception e) {
// Expected.
@ -731,8 +726,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
// Verify descriptor from HDFS
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
TableDescriptor td =
FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
verifyTableDescriptor(td, tableName, families);
}
@ -768,7 +762,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
@Test
public void testCompactionTimestamps() throws Exception {
createTableWithDefaultConf(tableName);
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
Optional<Long> ts = admin.getLastMajorCompactionTimestamp(tableName).get();
assertFalse(ts.isPresent());
Put p = new Put(Bytes.toBytes("row1"));
@ -783,9 +777,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
// still 0, we flushed a file, but no major compaction happened
assertFalse(ts.isPresent());
byte[] regionName =
ASYNC_CONN.getRegionLocator(tableName).getRegionLocation(Bytes.toBytes("row1")).get()
.getRegionInfo().getRegionName();
byte[] regionName = ASYNC_CONN.getRegionLocator(tableName)
.getRegionLocation(Bytes.toBytes("row1")).get().getRegion().getRegionName();
Optional<Long> ts1 = admin.getLastMajorCompactionTimestampForRegion(regionName).get();
assertFalse(ts1.isPresent());
p = new Put(Bytes.toBytes("row2"));
@ -823,7 +816,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
}
}
// Sleep to wait region server report
Thread.sleep(TEST_UTIL.getConfiguration().getInt("hbase.regionserver.msginterval", 3 * 1000) * 2);
Thread
.sleep(TEST_UTIL.getConfiguration().getInt("hbase.regionserver.msginterval", 3 * 1000) * 2);
ts = admin.getLastMajorCompactionTimestamp(tableName).get();
// after a compaction our earliest timestamp will have progressed forward

View File

@ -32,13 +32,15 @@ import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ForkJoinPool;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
@ -80,20 +82,20 @@ public class TestAsyncTableBatch {
public String tableType;
@Parameter(1)
public Function<TableName, AsyncTableBase> tableGetter;
public Function<TableName, AsyncTable<?>> tableGetter;
private static RawAsyncTable getRawTable(TableName tableName) {
return CONN.getRawTable(tableName);
private static AsyncTable<?> getRawTable(TableName tableName) {
return CONN.getTable(tableName);
}
private static AsyncTable getTable(TableName tableName) {
private static AsyncTable<?> getTable(TableName tableName) {
return CONN.getTable(tableName, ForkJoinPool.commonPool());
}
@Parameters(name = "{index}: type={0}")
public static List<Object[]> params() {
Function<TableName, AsyncTableBase> rawTableGetter = TestAsyncTableBatch::getRawTable;
Function<TableName, AsyncTableBase> tableGetter = TestAsyncTableBatch::getTable;
Function<TableName, AsyncTable<?>> rawTableGetter = TestAsyncTableBatch::getRawTable;
Function<TableName, AsyncTable<?>> tableGetter = TestAsyncTableBatch::getTable;
return Arrays.asList(new Object[] { "raw", rawTableGetter },
new Object[] { "normal", tableGetter });
}
@ -134,18 +136,15 @@ public class TestAsyncTableBatch {
}
@Test
public void test() throws InterruptedException, ExecutionException, IOException {
AsyncTableBase table = tableGetter.apply(TABLE_NAME);
public void test()
throws InterruptedException, ExecutionException, IOException, TimeoutException {
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
table.putAll(IntStream.range(0, COUNT)
.mapToObj(i -> new Put(getRow(i)).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
.collect(Collectors.toList())).get();
List<Result> results =
table
.getAll(IntStream.range(0, COUNT)
.mapToObj(
i -> Arrays.asList(new Get(getRow(i)), new Get(Arrays.copyOf(getRow(i), 4))))
.flatMap(l -> l.stream()).collect(Collectors.toList()))
.get();
List<Result> results = table.getAll(IntStream.range(0, COUNT)
.mapToObj(i -> Arrays.asList(new Get(getRow(i)), new Get(Arrays.copyOf(getRow(i), 4))))
.flatMap(l -> l.stream()).collect(Collectors.toList())).get();
assertEquals(2 * COUNT, results.size());
for (int i = 0; i < COUNT; i++) {
assertEquals(i, Bytes.toInt(results.get(2 * i).getValue(FAMILY, CQ)));
@ -153,19 +152,20 @@ public class TestAsyncTableBatch {
}
Admin admin = TEST_UTIL.getAdmin();
admin.flush(TABLE_NAME);
TEST_UTIL.getHBaseCluster().getRegions(TABLE_NAME).forEach(r -> {
byte[] startKey = r.getRegionInfo().getStartKey();
int number = startKey.length == 0 ? 55 : Integer.parseInt(Bytes.toString(startKey));
byte[] splitPoint = Bytes.toBytes(String.format("%03d", number + 55));
try {
admin.splitRegion(r.getRegionInfo().getRegionName(), splitPoint);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
});
// we are not going to test the function of split so no assertion here. Just wait for a while
// and then start our work.
Thread.sleep(5000);
List<Future<?>> splitFutures =
TEST_UTIL.getHBaseCluster().getRegions(TABLE_NAME).stream().map(r -> {
byte[] startKey = r.getRegionInfo().getStartKey();
int number = startKey.length == 0 ? 55 : Integer.parseInt(Bytes.toString(startKey));
byte[] splitPoint = Bytes.toBytes(String.format("%03d", number + 55));
try {
return admin.splitRegionAsync(r.getRegionInfo().getRegionName(), splitPoint);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}).collect(Collectors.toList());
for (Future<?> future : splitFutures) {
future.get(30, TimeUnit.SECONDS);
}
table.deleteAll(
IntStream.range(0, COUNT).mapToObj(i -> new Delete(getRow(i))).collect(Collectors.toList()))
.get();
@ -179,7 +179,7 @@ public class TestAsyncTableBatch {
@Test
public void testMixed() throws InterruptedException, ExecutionException {
AsyncTableBase table = tableGetter.apply(TABLE_NAME);
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
table.putAll(IntStream.range(0, 5)
.mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(FAMILY, CQ, Bytes.toBytes((long) i)))
.collect(Collectors.toList())).get();
@ -223,10 +223,10 @@ public class TestAsyncTableBatch {
@Test
public void testPartialSuccess() throws IOException, InterruptedException, ExecutionException {
Admin admin = TEST_UTIL.getAdmin();
HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLE_NAME));
htd.addCoprocessor(ErrorInjectObserver.class.getName());
admin.modifyTable(TABLE_NAME, htd);
AsyncTableBase table = tableGetter.apply(TABLE_NAME);
TableDescriptor htd = TableDescriptorBuilder.newBuilder(admin.getDescriptor(TABLE_NAME))
.addCoprocessor(ErrorInjectObserver.class.getName()).build();
admin.modifyTable(htd);
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
table.putAll(Arrays.asList(SPLIT_KEYS).stream().map(k -> new Put(k).addColumn(FAMILY, CQ, k))
.collect(Collectors.toList())).get();
List<CompletableFuture<Result>> futures = table

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT;
import static org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.TABLES_ON_MASTER;
import static org.apache.hadoop.hbase.master.LoadBalancer.TABLES_ON_MASTER;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
@ -38,7 +38,6 @@ import java.util.stream.IntStream;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -71,7 +70,7 @@ public class TestAsyncTableGetMultiThreaded {
private static AsyncConnection CONN;
private static RawAsyncTable TABLE;
private static AsyncTable<?> TABLE;
private static byte[][] SPLIT_KEYS;
@ -80,8 +79,7 @@ public class TestAsyncTableGetMultiThreaded {
setUp(MemoryCompactionPolicy.NONE);
}
protected static void setUp(MemoryCompactionPolicy memoryCompaction)
throws Exception {
protected static void setUp(MemoryCompactionPolicy memoryCompaction) throws Exception {
TEST_UTIL.getConfiguration().set(TABLES_ON_MASTER, "none");
TEST_UTIL.getConfiguration().setLong(HBASE_CLIENT_META_OPERATION_TIMEOUT, 60000L);
TEST_UTIL.getConfiguration().setInt(ByteBufferPool.MAX_POOL_SIZE_KEY, 100);
@ -96,7 +94,7 @@ public class TestAsyncTableGetMultiThreaded {
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
TEST_UTIL.waitTableAvailable(TABLE_NAME);
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
TABLE = CONN.getRawTableBuilder(TABLE_NAME).setReadRpcTimeout(1, TimeUnit.SECONDS)
TABLE = CONN.getTableBuilder(TABLE_NAME).setReadRpcTimeout(1, TimeUnit.SECONDS)
.setMaxRetries(1000).build();
TABLE.putAll(
IntStream.range(0, COUNT).mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
@ -123,8 +121,8 @@ public class TestAsyncTableGetMultiThreaded {
public void test() throws IOException, InterruptedException, ExecutionException {
int numThreads = 20;
AtomicBoolean stop = new AtomicBoolean(false);
ExecutorService executor = Executors.newFixedThreadPool(numThreads,
Threads.newDaemonThreadFactory("TestAsyncGet-"));
ExecutorService executor =
Executors.newFixedThreadPool(numThreads, Threads.newDaemonThreadFactory("TestAsyncGet-"));
List<Future<?>> futures = new ArrayList<>();
IntStream.range(0, numThreads).forEach(i -> futures.add(executor.submit(() -> {
run(stop);
@ -138,13 +136,13 @@ public class TestAsyncTableGetMultiThreaded {
region.compact(true);
}
Thread.sleep(5000);
admin.balancer(true);
admin.balance(true);
Thread.sleep(5000);
ServerName metaServer = TEST_UTIL.getHBaseCluster().getServerHoldingMeta();
ServerName newMetaServer = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
.map(t -> t.getRegionServer().getServerName()).filter(s -> !s.equals(metaServer))
.findAny().get();
admin.move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
admin.move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
Bytes.toBytes(newMetaServer.getServerName()));
Thread.sleep(5000);
}

View File

@ -102,7 +102,7 @@ public class TestAsyncTableNoncedRetry {
@Test
public void testAppend() throws InterruptedException, ExecutionException {
RawAsyncTable table = ASYNC_CONN.getRawTable(TABLE_NAME);
AsyncTable<?> table = ASYNC_CONN.getTable(TABLE_NAME);
Result result = table.append(new Append(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER));
result = table.append(new Append(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
@ -114,7 +114,7 @@ public class TestAsyncTableNoncedRetry {
@Test
public void testIncrement() throws InterruptedException, ExecutionException {
RawAsyncTable table = ASYNC_CONN.getRawTable(TABLE_NAME);
AsyncTable<?> table = ASYNC_CONN.getTable(TABLE_NAME);
assertEquals(1L, table.incrementColumnValue(row, FAMILY, QUALIFIER, 1L).get().longValue());
// the second call should have no effect as we always generate the same nonce.
assertEquals(1L, table.incrementColumnValue(row, FAMILY, QUALIFIER, 1L).get().longValue());

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
import java.util.List;
import java.util.concurrent.ForkJoinPool;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -42,8 +41,7 @@ public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
@Parameters(name = "{index}: scan={0}")
public static List<Object[]> params() {
return getScanCreater().stream().map(p -> new Object[] { p.getFirst(), p.getSecond() })
.collect(Collectors.toList());
return getScanCreatorParams();
}
@Override
@ -53,7 +51,8 @@ public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
@Override
protected List<Result> doScan(Scan scan) throws Exception {
AsyncTable table = ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
AsyncTable<ScanResultConsumer> table =
ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
SimpleScanResultConsumer consumer = new SimpleScanResultConsumer();
table.scan(scan, consumer);
List<Result> results = consumer.getAll();

View File

@ -17,11 +17,8 @@
*/
package org.apache.hadoop.hbase.client;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.ForkJoinPool;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -39,7 +36,7 @@ public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
public String tableType;
@Parameter(1)
public Supplier<AsyncTableBase> getTable;
public Supplier<AsyncTable<?>> getTable;
@Parameter(2)
public String scanType;
@ -47,22 +44,9 @@ public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
@Parameter(3)
public Supplier<Scan> scanCreator;
private static RawAsyncTable getRawTable() {
return ASYNC_CONN.getRawTable(TABLE_NAME);
}
private static AsyncTable getTable() {
return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
}
@Parameters(name = "{index}: table={0}, scan={2}")
public static List<Object[]> params() {
Supplier<AsyncTableBase> rawTable = TestAsyncTableScanAll::getRawTable;
Supplier<AsyncTableBase> normalTable = TestAsyncTableScanAll::getTable;
return getScanCreater().stream()
.flatMap(p -> Arrays.asList(new Object[] { "raw", rawTable, p.getFirst(), p.getSecond() },
new Object[] { "normal", normalTable, p.getFirst(), p.getSecond() }).stream())
.collect(Collectors.toList());
return getTableAndScanCreatorParams();
}
@Override

View File

@ -106,8 +106,8 @@ public class TestAsyncTableScanMetrics {
private static Pair<List<Result>, ScanMetrics> doScanWithRawAsyncTable(Scan scan)
throws IOException, InterruptedException {
SimpleRawScanResultConsumer consumer = new SimpleRawScanResultConsumer();
CONN.getRawTable(TABLE_NAME).scan(scan, consumer);
BufferingScanResultConsumer consumer = new BufferingScanResultConsumer();
CONN.getTable(TABLE_NAME).scan(scan, consumer);
List<Result> results = new ArrayList<>();
for (Result result; (result = consumer.take()) != null;) {
results.add(result);

View File

@ -49,7 +49,7 @@ public class TestAsyncTableScanRenewLease {
private static AsyncConnection CONN;
private static RawAsyncTable TABLE;
private static AsyncTable<AdvancedScanResultConsumer> TABLE;
private static int SCANNER_LEASE_TIMEOUT_PERIOD_MS = 5000;
@ -60,7 +60,7 @@ public class TestAsyncTableScanRenewLease {
TEST_UTIL.startMiniCluster(1);
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
TABLE = CONN.getRawTable(TABLE_NAME);
TABLE = CONN.getTable(TABLE_NAME);
TABLE.putAll(IntStream.range(0, 10).mapToObj(
i -> new Put(Bytes.toBytes(String.format("%02d", i))).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
.collect(Collectors.toList())).get();
@ -72,7 +72,7 @@ public class TestAsyncTableScanRenewLease {
TEST_UTIL.shutdownMiniCluster();
}
private static final class RenewLeaseConsumer implements RawScanResultConsumer {
private static final class RenewLeaseConsumer implements AdvancedScanResultConsumer {
private final List<Result> results = new ArrayList<>();

View File

@ -21,7 +21,6 @@ import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ForkJoinPool;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -36,25 +35,30 @@ import org.junit.runners.Parameterized.Parameters;
public class TestAsyncTableScanner extends AbstractTestAsyncTableScan {
@Parameter(0)
public String scanType;
public String tableType;
@Parameter(1)
public Supplier<Scan> scanCreater;
public Supplier<AsyncTable<?>> getTable;
@Parameters(name = "{index}: scan={0}")
@Parameter(2)
public String scanType;
@Parameter(3)
public Supplier<Scan> scanCreator;
@Parameters(name = "{index}: table={0}, scan={2}")
public static List<Object[]> params() {
return getScanCreater().stream().map(p -> new Object[] { p.getFirst(), p.getSecond() })
.collect(Collectors.toList());
return getTableAndScanCreatorParams();
}
@Override
protected Scan createScan() {
return scanCreater.get();
return scanCreator.get();
}
@Override
protected List<Result> doScan(Scan scan) throws Exception {
AsyncTable table = ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
AsyncTable<?> table = ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
List<Result> results = new ArrayList<>();
try (ResultScanner scanner = table.getScanner(scan)) {
for (Result result; (result = scanner.next()) != null;) {

View File

@ -47,7 +47,7 @@ public class TestAsyncTableScannerCloseWhileSuspending {
private static AsyncConnection CONN;
private static AsyncTable TABLE;
private static AsyncTable<?> TABLE;
@BeforeClass
public static void setUp() throws Exception {

View File

@ -52,9 +52,9 @@ public class TestRawAsyncScanCursor extends AbstractTestScanCursor {
private void doTest(boolean reversed)
throws InterruptedException, ExecutionException, IOException {
CompletableFuture<Void> future = new CompletableFuture<>();
RawAsyncTable table = CONN.getRawTable(TABLE_NAME);
AsyncTable<AdvancedScanResultConsumer> table = CONN.getTable(TABLE_NAME);
table.scan(reversed ? createReversedScanWithSparseFilter() : createScanWithSparseFilter(),
new RawScanResultConsumer() {
new AdvancedScanResultConsumer() {
private int count;
@ -121,8 +121,8 @@ public class TestRawAsyncScanCursor extends AbstractTestScanCursor {
@Test
public void testSizeLimit() throws InterruptedException, ExecutionException {
CompletableFuture<Void> future = new CompletableFuture<>();
RawAsyncTable table = CONN.getRawTable(TABLE_NAME);
table.scan(createScanWithSizeLimit(), new RawScanResultConsumer() {
AsyncTable<AdvancedScanResultConsumer> table = CONN.getTable(TABLE_NAME);
table.scan(createScanWithSizeLimit(), new AdvancedScanResultConsumer() {
private int count;

View File

@ -58,14 +58,14 @@ public class TestRawAsyncTableLimitedScanWithFilter {
private static AsyncConnection CONN;
private static RawAsyncTable TABLE;
private static AsyncTable<?> TABLE;
@BeforeClass
public static void setUp() throws Exception {
UTIL.startMiniCluster(1);
UTIL.createTable(TABLE_NAME, FAMILY);
CONN = ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get();
TABLE = CONN.getRawTable(TABLE_NAME);
TABLE = CONN.getTable(TABLE_NAME);
TABLE.putAll(IntStream.range(0, ROW_COUNT).mapToObj(i -> {
Put put = new Put(Bytes.toBytes(i));
IntStream.range(0, CQS.length)

View File

@ -44,13 +44,13 @@ public class TestRawAsyncTablePartialScan {
private static byte[] FAMILY = Bytes.toBytes("cf");
private static byte[][] CQS =
new byte[][] { Bytes.toBytes("cq1"), Bytes.toBytes("cq2"), Bytes.toBytes("cq3") };
new byte[][] { Bytes.toBytes("cq1"), Bytes.toBytes("cq2"), Bytes.toBytes("cq3") };
private static int COUNT = 100;
private static AsyncConnection CONN;
private static RawAsyncTable TABLE;
private static AsyncTable<?> TABLE;
@BeforeClass
public static void setUp() throws Exception {
@ -58,7 +58,7 @@ public class TestRawAsyncTablePartialScan {
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
TEST_UTIL.waitTableAvailable(TABLE_NAME);
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
TABLE = CONN.getRawTable(TABLE_NAME);
TABLE = CONN.getTable(TABLE_NAME);
TABLE
.putAll(IntStream.range(0, COUNT)
.mapToObj(i -> new Put(Bytes.toBytes(String.format("%02d", i)))
@ -100,7 +100,7 @@ public class TestRawAsyncTablePartialScan {
// we set batch to 2 and max result size to 1, then server will only returns one result per call
// but we should get 2 + 1 for every row.
List<Result> results =
TABLE.scanAll(new Scan().setBatch(2).setMaxResultSize(1).setReversed(true)).get();
TABLE.scanAll(new Scan().setBatch(2).setMaxResultSize(1).setReversed(true)).get();
assertEquals(2 * COUNT, results.size());
for (int i = 0; i < COUNT; i++) {
int row = COUNT - i - 1;

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -42,8 +41,7 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
@Parameters(name = "{index}: type={0}")
public static List<Object[]> params() {
return getScanCreater().stream().map(p -> new Object[] { p.getFirst(), p.getSecond() })
.collect(Collectors.toList());
return getScanCreatorParams();
}
@Override
@ -53,8 +51,8 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
@Override
protected List<Result> doScan(Scan scan) throws Exception {
SimpleRawScanResultConsumer scanConsumer = new SimpleRawScanResultConsumer();
ASYNC_CONN.getRawTable(TABLE_NAME).scan(scan, scanConsumer);
BufferingScanResultConsumer scanConsumer = new BufferingScanResultConsumer();
ASYNC_CONN.getTable(TABLE_NAME).scan(scan, scanConsumer);
List<Result> results = new ArrayList<>();
for (Result result; (result = scanConsumer.take()) != null;) {
results.add(result);