HBASE-19251 Merge RawAsyncTable and AsyncTable
This commit is contained in:
parent
3a4655019d
commit
54827cf613
@ -38,10 +38,10 @@ import org.apache.commons.logging.LogFactory;
|
|||||||
import org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor;
|
import org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor;
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
|
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
|
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.Consistency;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
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.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
@ -72,14 +72,15 @@ public class AsyncMetaTableAccessor {
|
|||||||
private static final Pattern SERVER_COLUMN_PATTERN = Pattern
|
private static final Pattern SERVER_COLUMN_PATTERN = Pattern
|
||||||
.compile("^server(_[0-9a-fA-F]{4})?$");
|
.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)) {
|
if (tableName.equals(META_TABLE_NAME)) {
|
||||||
return CompletableFuture.completedFuture(true);
|
return CompletableFuture.completedFuture(true);
|
||||||
}
|
}
|
||||||
return getTableState(metaTable, tableName).thenApply(Optional::isPresent);
|
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) {
|
TableName tableName) {
|
||||||
CompletableFuture<Optional<TableState>> future = new CompletableFuture<>();
|
CompletableFuture<Optional<TableState>> future = new CompletableFuture<>();
|
||||||
Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
|
Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
|
||||||
@ -110,7 +111,7 @@ public class AsyncMetaTableAccessor {
|
|||||||
* @return HRegionLocation for the given region
|
* @return HRegionLocation for the given region
|
||||||
*/
|
*/
|
||||||
public static CompletableFuture<Optional<HRegionLocation>> getRegionLocation(
|
public static CompletableFuture<Optional<HRegionLocation>> getRegionLocation(
|
||||||
RawAsyncTable metaTable, byte[] regionName) {
|
AsyncTable<?> metaTable, byte[] regionName) {
|
||||||
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
|
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
|
||||||
try {
|
try {
|
||||||
RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
|
RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
|
||||||
@ -139,7 +140,7 @@ public class AsyncMetaTableAccessor {
|
|||||||
* @return HRegionLocation for the given region
|
* @return HRegionLocation for the given region
|
||||||
*/
|
*/
|
||||||
public static CompletableFuture<Optional<HRegionLocation>> getRegionLocationWithEncodedName(
|
public static CompletableFuture<Optional<HRegionLocation>> getRegionLocationWithEncodedName(
|
||||||
RawAsyncTable metaTable, byte[] encodedRegionName) {
|
AsyncTable<?> metaTable, byte[] encodedRegionName) {
|
||||||
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
|
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
|
||||||
metaTable.scanAll(new Scan().setReadType(ReadType.PREAD).addFamily(HConstants.CATALOG_FAMILY))
|
metaTable.scanAll(new Scan().setReadType(ReadType.PREAD).addFamily(HConstants.CATALOG_FAMILY))
|
||||||
.whenComplete(
|
.whenComplete(
|
||||||
@ -193,7 +194,7 @@ public class AsyncMetaTableAccessor {
|
|||||||
* {@link CompletableFuture}.
|
* {@link CompletableFuture}.
|
||||||
*/
|
*/
|
||||||
public static CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(
|
public static CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(
|
||||||
RawAsyncTable metaTable, final Optional<TableName> tableName) {
|
AsyncTable<AdvancedScanResultConsumer> metaTable, Optional<TableName> tableName) {
|
||||||
CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
|
CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
|
||||||
getTableRegionsAndLocations(metaTable, tableName, true).whenComplete(
|
getTableRegionsAndLocations(metaTable, tableName, true).whenComplete(
|
||||||
(locations, err) -> {
|
(locations, err) -> {
|
||||||
@ -220,7 +221,7 @@ public class AsyncMetaTableAccessor {
|
|||||||
* {@link CompletableFuture}.
|
* {@link CompletableFuture}.
|
||||||
*/
|
*/
|
||||||
private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
|
private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
|
||||||
RawAsyncTable metaTable, final Optional<TableName> tableName,
|
AsyncTable<AdvancedScanResultConsumer> metaTable, final Optional<TableName> tableName,
|
||||||
final boolean excludeOfflinedSplitParents) {
|
final boolean excludeOfflinedSplitParents) {
|
||||||
CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
|
CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
|
||||||
if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) {
|
if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) {
|
||||||
@ -252,7 +253,7 @@ public class AsyncMetaTableAccessor {
|
|||||||
}
|
}
|
||||||
for (HRegionLocation loc : current.get().getRegionLocations()) {
|
for (HRegionLocation loc : current.get().getRegionLocations()) {
|
||||||
if (loc != null) {
|
if (loc != null) {
|
||||||
this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegionInfo(), loc
|
this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegion(), loc
|
||||||
.getServerName()));
|
.getServerName()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -276,7 +277,7 @@ public class AsyncMetaTableAccessor {
|
|||||||
* @param type scanned part of meta
|
* @param type scanned part of meta
|
||||||
* @param visitor Visitor invoked against each row
|
* @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) {
|
Optional<TableName> tableName, QueryType type, final Visitor visitor) {
|
||||||
return scanMeta(metaTable, getTableStartRowForMeta(tableName, type),
|
return scanMeta(metaTable, getTableStartRowForMeta(tableName, type),
|
||||||
getTableStopRowForMeta(tableName, type), type, Integer.MAX_VALUE, visitor);
|
getTableStopRowForMeta(tableName, type), type, Integer.MAX_VALUE, visitor);
|
||||||
@ -291,8 +292,9 @@ public class AsyncMetaTableAccessor {
|
|||||||
* @param maxRows maximum rows to return
|
* @param maxRows maximum rows to return
|
||||||
* @param visitor Visitor invoked against each row
|
* @param visitor Visitor invoked against each row
|
||||||
*/
|
*/
|
||||||
private static CompletableFuture<Void> scanMeta(RawAsyncTable metaTable, Optional<byte[]> startRow,
|
private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable,
|
||||||
Optional<byte[]> stopRow, QueryType type, int maxRows, final Visitor visitor) {
|
Optional<byte[]> startRow, Optional<byte[]> stopRow, QueryType type, int maxRows,
|
||||||
|
final Visitor visitor) {
|
||||||
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
|
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
|
||||||
Scan scan = getMetaScan(metaTable, rowUpperLimit);
|
Scan scan = getMetaScan(metaTable, rowUpperLimit);
|
||||||
for (byte[] family : type.getFamilies()) {
|
for (byte[] family : type.getFamilies()) {
|
||||||
@ -308,11 +310,11 @@ public class AsyncMetaTableAccessor {
|
|||||||
}
|
}
|
||||||
|
|
||||||
CompletableFuture<Void> future = new CompletableFuture<Void>();
|
CompletableFuture<Void> future = new CompletableFuture<Void>();
|
||||||
metaTable.scan(scan, new MetaTableRawScanResultConsumer(rowUpperLimit, visitor, future));
|
metaTable.scan(scan, new MetaTableScanResultConsumer(rowUpperLimit, visitor, future));
|
||||||
return future;
|
return future;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final class MetaTableRawScanResultConsumer implements RawScanResultConsumer {
|
private static final class MetaTableScanResultConsumer implements AdvancedScanResultConsumer {
|
||||||
|
|
||||||
private int currentRowCount;
|
private int currentRowCount;
|
||||||
|
|
||||||
@ -322,7 +324,8 @@ public class AsyncMetaTableAccessor {
|
|||||||
|
|
||||||
private final CompletableFuture<Void> future;
|
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.rowUpperLimit = rowUpperLimit;
|
||||||
this.visitor = visitor;
|
this.visitor = visitor;
|
||||||
this.future = future;
|
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();
|
Scan scan = new Scan();
|
||||||
int scannerCaching = metaTable.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
|
int scannerCaching = metaTable.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
|
||||||
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
|
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
|
||||||
|
@ -20,21 +20,26 @@ package org.apache.hadoop.hbase.client;
|
|||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
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>
|
* <p>
|
||||||
* Notice that, the {@link #onNext(Result[], ScanController)} method will be called in the thread
|
* All results that match the given scan object will be passed to this class by calling
|
||||||
* which we send request to HBase service. So if you want the asynchronous scanner fetch data from
|
* {@link #onNext(Result[], ScanController)}. {@link #onComplete()} means the scan is finished, and
|
||||||
* HBase in background while you process the returned data, you need to move the processing work to
|
* {@link #onError(Throwable)} means we hit an unrecoverable error and the scan is terminated.
|
||||||
* another thread to make the {@code onNext} call return immediately. And please do NOT do any time
|
* {@link #onHeartbeat(ScanController)} means the RS is still working but we can not get a valid
|
||||||
* consuming tasks in all methods below unless you know what you are doing.
|
* 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
|
* @since 2.0.0
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Public
|
@InterfaceAudience.Public
|
||||||
public interface RawScanResultConsumer {
|
public interface AdvancedScanResultConsumer extends ScanResultConsumerBase {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Used to resume a scan.
|
* Used to resume a scan.
|
||||||
@ -113,25 +118,4 @@ public interface RawScanResultConsumer {
|
|||||||
*/
|
*/
|
||||||
default void onHeartbeat(ScanController controller) {
|
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) {
|
|
||||||
}
|
|
||||||
}
|
}
|
@ -17,6 +17,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
|
import com.google.protobuf.RpcChannel;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.List;
|
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.RegionLoad;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
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.replication.TableCFs;
|
||||||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||||
import org.apache.hadoop.hbase.quotas.QuotaFilter;
|
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.hadoop.hbase.replication.ReplicationPeerDescription;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
import com.google.protobuf.RpcChannel;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The asynchronous administrative API for HBase.
|
* The asynchronous administrative API for HBase.
|
||||||
* @since 2.0.0
|
* @since 2.0.0
|
||||||
@ -1072,14 +1071,14 @@ public interface AsyncAdmin {
|
|||||||
* </pre>
|
* </pre>
|
||||||
* @param stubMaker a delegation to the actual {@code newStub} call.
|
* @param stubMaker a delegation to the actual {@code newStub} call.
|
||||||
* @param callable a delegation to the actual protobuf rpc call. See the comment of
|
* @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 <S> the type of the asynchronous stub
|
||||||
* @param <R> the type of the return value
|
* @param <R> the type of the return value
|
||||||
* @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
|
* @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,
|
<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.
|
* Execute the given coprocessor call on the given region server.
|
||||||
@ -1094,15 +1093,15 @@ public interface AsyncAdmin {
|
|||||||
* </pre>
|
* </pre>
|
||||||
* @param stubMaker a delegation to the actual {@code newStub} call.
|
* @param stubMaker a delegation to the actual {@code newStub} call.
|
||||||
* @param callable a delegation to the actual protobuf rpc call. See the comment of
|
* @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 serverName the given region server
|
||||||
* @param <S> the type of the asynchronous stub
|
* @param <S> the type of the asynchronous stub
|
||||||
* @param <R> the type of the return value
|
* @param <R> the type of the return value
|
||||||
* @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
|
* @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,
|
<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.
|
* List all the dead region servers.
|
||||||
|
@ -29,12 +29,12 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
class AsyncBufferedMutatorBuilderImpl implements AsyncBufferedMutatorBuilder {
|
class AsyncBufferedMutatorBuilderImpl implements AsyncBufferedMutatorBuilder {
|
||||||
|
|
||||||
private final AsyncTableBuilder<? extends AsyncTableBase> tableBuilder;
|
private final AsyncTableBuilder<?> tableBuilder;
|
||||||
|
|
||||||
private long writeBufferSize;
|
private long writeBufferSize;
|
||||||
|
|
||||||
public AsyncBufferedMutatorBuilderImpl(AsyncConnectionConfiguration connConf,
|
public AsyncBufferedMutatorBuilderImpl(AsyncConnectionConfiguration connConf,
|
||||||
AsyncTableBuilder<? extends AsyncTableBase> tableBuilder) {
|
AsyncTableBuilder<?> tableBuilder) {
|
||||||
this.tableBuilder = tableBuilder;
|
this.tableBuilder = tableBuilder;
|
||||||
this.writeBufferSize = connConf.getWriteBufferSize();
|
this.writeBufferSize = connConf.getWriteBufferSize();
|
||||||
}
|
}
|
||||||
|
@ -30,12 +30,12 @@ import org.apache.hadoop.hbase.TableName;
|
|||||||
import org.apache.yetus.audience.InterfaceAudience;
|
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
|
@InterfaceAudience.Private
|
||||||
class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
|
class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
|
||||||
|
|
||||||
private final AsyncTableBase table;
|
private final AsyncTable<?> table;
|
||||||
|
|
||||||
private final long writeBufferSize;
|
private final long writeBufferSize;
|
||||||
|
|
||||||
@ -47,7 +47,7 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
|
|||||||
|
|
||||||
private boolean closed;
|
private boolean closed;
|
||||||
|
|
||||||
AsyncBufferedMutatorImpl(AsyncTableBase table, long writeBufferSize) {
|
AsyncBufferedMutatorImpl(AsyncTable<?> table, long writeBufferSize) {
|
||||||
this.table = table;
|
this.table = table;
|
||||||
this.writeBufferSize = writeBufferSize;
|
this.writeBufferSize = writeBufferSize;
|
||||||
}
|
}
|
||||||
|
@ -53,7 +53,7 @@ class AsyncClientScanner {
|
|||||||
|
|
||||||
private final ScanMetrics scanMetrics;
|
private final ScanMetrics scanMetrics;
|
||||||
|
|
||||||
private final RawScanResultConsumer consumer;
|
private final AdvancedScanResultConsumer consumer;
|
||||||
|
|
||||||
private final TableName tableName;
|
private final TableName tableName;
|
||||||
|
|
||||||
@ -71,7 +71,7 @@ class AsyncClientScanner {
|
|||||||
|
|
||||||
private final ScanResultCache resultCache;
|
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,
|
AsyncConnectionImpl conn, long pauseNs, int maxAttempts, long scanTimeoutNs,
|
||||||
long rpcTimeoutNs, int startLogErrorsCnt) {
|
long rpcTimeoutNs, int startLogErrorsCnt) {
|
||||||
if (scan.getStartRow() == null) {
|
if (scan.getStartRow() == null) {
|
||||||
|
@ -49,32 +49,37 @@ public interface AsyncConnection extends Closeable {
|
|||||||
AsyncTableRegionLocator getRegionLocator(TableName tableName);
|
AsyncTableRegionLocator getRegionLocator(TableName tableName);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve an {@link RawAsyncTable} implementation for accessing a table.
|
* Retrieve an {@link AsyncTable} implementation for accessing a table.
|
||||||
* <p>
|
* <p>
|
||||||
* The returned instance will use default configs. Use {@link #getRawTableBuilder(TableName)} if you
|
* The returned instance will use default configs. Use {@link #getTableBuilder(TableName)} if
|
||||||
* want to customize some configs.
|
* you want to customize some configs.
|
||||||
* <p>
|
* <p>
|
||||||
* This method no longer checks table existence. An exception will be thrown if the table does not
|
* 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.
|
* 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
|
* @param tableName the name of the table
|
||||||
* @return an RawAsyncTable to use for interactions with this table
|
* @return an AsyncTable to use for interactions with this table
|
||||||
* @see #getRawTableBuilder(TableName)
|
* @see #getTableBuilder(TableName)
|
||||||
*/
|
*/
|
||||||
default RawAsyncTable getRawTable(TableName tableName) {
|
default AsyncTable<AdvancedScanResultConsumer> getTable(TableName tableName) {
|
||||||
return getRawTableBuilder(tableName).build();
|
return getTableBuilder(tableName).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns an {@link AsyncTableBuilder} for creating {@link RawAsyncTable}.
|
* Returns an {@link AsyncTableBuilder} for creating {@link AsyncTable}.
|
||||||
* <p>
|
* <p>
|
||||||
* This method no longer checks table existence. An exception will be thrown if the table does not
|
* 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.
|
* exist only when the first operation is attempted.
|
||||||
* @param tableName the name of the table
|
* @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>
|
* <p>
|
||||||
* This method no longer checks table existence. An exception will be thrown if the table does not
|
* 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.
|
* 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
|
* @param pool the thread pool to use for executing callback
|
||||||
* @return an AsyncTable to use for interactions with this table
|
* @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();
|
return getTableBuilder(tableName, pool).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -94,7 +99,7 @@ public interface AsyncConnection extends Closeable {
|
|||||||
* @param tableName the name of the table
|
* @param tableName the name of the table
|
||||||
* @param pool the thread pool to use for executing callback
|
* @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.
|
* Retrieve an {@link AsyncAdmin} implementation to administer an HBase cluster.
|
||||||
|
@ -97,7 +97,7 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
private final AtomicReference<MasterService.Interface> masterStub = new AtomicReference<>();
|
private final AtomicReference<MasterService.Interface> masterStub = new AtomicReference<>();
|
||||||
|
|
||||||
private final AtomicReference<CompletableFuture<MasterService.Interface>> masterStubMakeFuture =
|
private final AtomicReference<CompletableFuture<MasterService.Interface>> masterStubMakeFuture =
|
||||||
new AtomicReference<>();
|
new AtomicReference<>();
|
||||||
|
|
||||||
public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
|
public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
|
||||||
User user) {
|
User user) {
|
||||||
@ -108,8 +108,8 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
this.rpcClient = RpcClientFactory.createClient(conf, clusterId);
|
this.rpcClient = RpcClientFactory.createClient(conf, clusterId);
|
||||||
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
|
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
|
||||||
this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
|
this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
|
||||||
this.rpcTimeout = (int) Math.min(Integer.MAX_VALUE,
|
this.rpcTimeout =
|
||||||
TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs()));
|
(int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs()));
|
||||||
this.locator = new AsyncRegionLocator(this, RETRY_TIMER);
|
this.locator = new AsyncRegionLocator(this, RETRY_TIMER);
|
||||||
this.callerFactory = new AsyncRpcRetryingCallerFactory(this, RETRY_TIMER);
|
this.callerFactory = new AsyncRpcRetryingCallerFactory(this, RETRY_TIMER);
|
||||||
if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) {
|
if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) {
|
||||||
@ -161,7 +161,7 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
return MasterService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
|
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));
|
return AdminService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -172,38 +172,37 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void makeMasterStub(CompletableFuture<MasterService.Interface> future) {
|
private void makeMasterStub(CompletableFuture<MasterService.Interface> future) {
|
||||||
registry.getMasterAddress().whenComplete(
|
registry.getMasterAddress().whenComplete((sn, error) -> {
|
||||||
(sn, error) -> {
|
if (sn == null) {
|
||||||
if (sn == null) {
|
String msg = "ZooKeeper available but no active master location found";
|
||||||
String msg = "ZooKeeper available but no active master location found";
|
LOG.info(msg);
|
||||||
LOG.info(msg);
|
this.masterStubMakeFuture.getAndSet(null)
|
||||||
this.masterStubMakeFuture.getAndSet(null).completeExceptionally(
|
.completeExceptionally(new MasterNotRunningException(msg));
|
||||||
new MasterNotRunningException(msg));
|
return;
|
||||||
return;
|
}
|
||||||
}
|
try {
|
||||||
try {
|
MasterService.Interface stub = createMasterStub(sn);
|
||||||
MasterService.Interface stub = createMasterStub(sn);
|
HBaseRpcController controller = getRpcController();
|
||||||
HBaseRpcController controller = getRpcController();
|
stub.isMasterRunning(controller, RequestConverter.buildIsMasterRunningRequest(),
|
||||||
stub.isMasterRunning(controller, RequestConverter.buildIsMasterRunningRequest(),
|
new RpcCallback<IsMasterRunningResponse>() {
|
||||||
new RpcCallback<IsMasterRunningResponse>() {
|
@Override
|
||||||
@Override
|
public void run(IsMasterRunningResponse resp) {
|
||||||
public void run(IsMasterRunningResponse resp) {
|
if (controller.failed() || resp == null ||
|
||||||
if (controller.failed() || resp == null
|
(resp != null && !resp.getIsMasterRunning())) {
|
||||||
|| (resp != null && !resp.getIsMasterRunning())) {
|
masterStubMakeFuture.getAndSet(null).completeExceptionally(
|
||||||
masterStubMakeFuture.getAndSet(null).completeExceptionally(
|
new MasterNotRunningException("Master connection is not running anymore"));
|
||||||
new MasterNotRunningException("Master connection is not running anymore"));
|
} else {
|
||||||
} else {
|
masterStub.set(stub);
|
||||||
masterStub.set(stub);
|
masterStubMakeFuture.set(null);
|
||||||
masterStubMakeFuture.set(null);
|
future.complete(stub);
|
||||||
future.complete(stub);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
});
|
}
|
||||||
} catch (IOException e) {
|
});
|
||||||
this.masterStubMakeFuture.getAndSet(null).completeExceptionally(
|
} catch (IOException e) {
|
||||||
new IOException("Failed to create async master stub", e));
|
this.masterStubMakeFuture.getAndSet(null)
|
||||||
}
|
.completeExceptionally(new IOException("Failed to create async master stub", e));
|
||||||
});
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
CompletableFuture<MasterService.Interface> getMasterStub() {
|
CompletableFuture<MasterService.Interface> getMasterStub() {
|
||||||
@ -231,8 +230,8 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
new RpcCallback<IsMasterRunningResponse>() {
|
new RpcCallback<IsMasterRunningResponse>() {
|
||||||
@Override
|
@Override
|
||||||
public void run(IsMasterRunningResponse resp) {
|
public void run(IsMasterRunningResponse resp) {
|
||||||
if (controller.failed() || resp == null
|
if (controller.failed() || resp == null ||
|
||||||
|| (resp != null && !resp.getIsMasterRunning())) {
|
(resp != null && !resp.getIsMasterRunning())) {
|
||||||
makeMasterStub(future);
|
makeMasterStub(future);
|
||||||
} else {
|
} else {
|
||||||
future.complete(masterStub);
|
future.complete(masterStub);
|
||||||
@ -255,22 +254,23 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilder<RawAsyncTable> getRawTableBuilder(TableName tableName) {
|
public AsyncTableBuilder<AdvancedScanResultConsumer> getTableBuilder(TableName tableName) {
|
||||||
return new AsyncTableBuilderBase<RawAsyncTable>(tableName, connConf) {
|
return new AsyncTableBuilderBase<AdvancedScanResultConsumer>(tableName, connConf) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RawAsyncTable build() {
|
public AsyncTable<AdvancedScanResultConsumer> build() {
|
||||||
return new RawAsyncTableImpl(AsyncConnectionImpl.this, this);
|
return new RawAsyncTableImpl(AsyncConnectionImpl.this, this);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilder<AsyncTable> getTableBuilder(TableName tableName, ExecutorService pool) {
|
public AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName,
|
||||||
return new AsyncTableBuilderBase<AsyncTable>(tableName, connConf) {
|
ExecutorService pool) {
|
||||||
|
return new AsyncTableBuilderBase<ScanResultConsumer>(tableName, connConf) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTable build() {
|
public AsyncTable<ScanResultConsumer> build() {
|
||||||
RawAsyncTableImpl rawTable = new RawAsyncTableImpl(AsyncConnectionImpl.this, this);
|
RawAsyncTableImpl rawTable = new RawAsyncTableImpl(AsyncConnectionImpl.this, this);
|
||||||
return new AsyncTableImpl(AsyncConnectionImpl.this, rawTable, pool);
|
return new AsyncTableImpl(AsyncConnectionImpl.this, rawTable, pool);
|
||||||
}
|
}
|
||||||
@ -293,7 +293,7 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
@Override
|
@Override
|
||||||
public AsyncAdmin build() {
|
public AsyncAdmin build() {
|
||||||
RawAsyncHBaseAdmin rawAdmin =
|
RawAsyncHBaseAdmin rawAdmin =
|
||||||
new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
|
new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
|
||||||
return new AsyncHBaseAdmin(rawAdmin, pool);
|
return new AsyncHBaseAdmin(rawAdmin, pool);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -301,7 +301,7 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {
|
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {
|
||||||
return new AsyncBufferedMutatorBuilderImpl(connConf, getRawTableBuilder(tableName));
|
return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|||||||
import org.apache.hadoop.hbase.RegionLoad;
|
import org.apache.hadoop.hbase.RegionLoad;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
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.replication.TableCFs;
|
||||||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||||
import org.apache.hadoop.hbase.quotas.QuotaFilter;
|
import org.apache.hadoop.hbase.quotas.QuotaFilter;
|
||||||
@ -56,7 +55,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||||||
* @see AsyncConnection#getAdminBuilder(ExecutorService)
|
* @see AsyncConnection#getAdminBuilder(ExecutorService)
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class AsyncHBaseAdmin implements AsyncAdmin {
|
class AsyncHBaseAdmin implements AsyncAdmin {
|
||||||
|
|
||||||
private final RawAsyncHBaseAdmin rawAdmin;
|
private final RawAsyncHBaseAdmin rawAdmin;
|
||||||
|
|
||||||
@ -705,13 +704,13 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
||||||
CoprocessorCallable<S, R> callable) {
|
ServiceCaller<S, R> callable) {
|
||||||
return wrap(rawAdmin.coprocessorService(stubMaker, callable));
|
return wrap(rawAdmin.coprocessorService(stubMaker, callable));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
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));
|
return wrap(rawAdmin.coprocessorService(stubMaker, callable, serverName));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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.CATALOG_FAMILY;
|
||||||
import static org.apache.hadoop.hbase.HConstants.NINES;
|
import static org.apache.hadoop.hbase.HConstants.NINES;
|
||||||
import static org.apache.hadoop.hbase.HConstants.ZEROES;
|
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.TableName.META_TABLE_NAME;
|
||||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
|
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.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.Bytes.BYTES_COMPARATOR;
|
||||||
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
|
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.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
|
||||||
import org.apache.hadoop.hbase.HRegionLocation;
|
import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.RegionLocations;
|
import org.apache.hadoop.hbase.RegionLocations;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The asynchronous locator for regions other than meta.
|
* The asynchronous locator for regions other than meta.
|
||||||
@ -63,7 +62,7 @@ class AsyncNonMetaRegionLocator {
|
|||||||
private static final Log LOG = LogFactory.getLog(AsyncNonMetaRegionLocator.class);
|
private static final Log LOG = LogFactory.getLog(AsyncNonMetaRegionLocator.class);
|
||||||
|
|
||||||
static final String MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE =
|
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;
|
private static final int DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE = 8;
|
||||||
|
|
||||||
@ -102,12 +101,12 @@ class AsyncNonMetaRegionLocator {
|
|||||||
private static final class TableCache {
|
private static final class TableCache {
|
||||||
|
|
||||||
public final ConcurrentNavigableMap<byte[], HRegionLocation> cache =
|
public final ConcurrentNavigableMap<byte[], HRegionLocation> cache =
|
||||||
new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
|
new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
|
||||||
|
|
||||||
public final Set<LocateRequest> pendingRequests = new HashSet<>();
|
public final Set<LocateRequest> pendingRequests = new HashSet<>();
|
||||||
|
|
||||||
public final Map<LocateRequest, CompletableFuture<HRegionLocation>> allRequests =
|
public final Map<LocateRequest, CompletableFuture<HRegionLocation>> allRequests =
|
||||||
new LinkedHashMap<>();
|
new LinkedHashMap<>();
|
||||||
|
|
||||||
public boolean hasQuota(int max) {
|
public boolean hasQuota(int max) {
|
||||||
return pendingRequests.size() < max;
|
return pendingRequests.size() < max;
|
||||||
@ -126,8 +125,8 @@ class AsyncNonMetaRegionLocator {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public void clearCompletedRequests(Optional<HRegionLocation> location) {
|
public void clearCompletedRequests(Optional<HRegionLocation> location) {
|
||||||
for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter = allRequests
|
for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter =
|
||||||
.entrySet().iterator(); iter.hasNext();) {
|
allRequests.entrySet().iterator(); iter.hasNext();) {
|
||||||
Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>> entry = iter.next();
|
Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>> entry = iter.next();
|
||||||
if (tryComplete(entry.getKey(), entry.getValue(), location)) {
|
if (tryComplete(entry.getKey(), entry.getValue(), location)) {
|
||||||
iter.remove();
|
iter.remove();
|
||||||
@ -146,15 +145,16 @@ class AsyncNonMetaRegionLocator {
|
|||||||
HRegionLocation loc = location.get();
|
HRegionLocation loc = location.get();
|
||||||
boolean completed;
|
boolean completed;
|
||||||
if (req.locateType.equals(RegionLocateType.BEFORE)) {
|
if (req.locateType.equals(RegionLocateType.BEFORE)) {
|
||||||
// for locating the row before current row, the common case is to find the previous region in
|
// for locating the row before current row, the common case is to find the previous region
|
||||||
// reverse scan, so we check the endKey first. In general, the condition should be startKey <
|
// in reverse scan, so we check the endKey first. In general, the condition should be
|
||||||
// req.row and endKey >= req.row. Here we split it to endKey == req.row || (endKey > req.row
|
// startKey < req.row and endKey >= req.row. Here we split it to endKey == req.row ||
|
||||||
// && startKey < req.row). The two conditions are equal since startKey < endKey.
|
// (endKey > req.row && startKey < req.row). The two conditions are equal since startKey <
|
||||||
int c = Bytes.compareTo(loc.getRegionInfo().getEndKey(), req.row);
|
// endKey.
|
||||||
|
int c = Bytes.compareTo(loc.getRegion().getEndKey(), req.row);
|
||||||
completed =
|
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 {
|
} else {
|
||||||
completed = loc.getRegionInfo().containsRow(req.row);
|
completed = loc.getRegion().containsRow(req.row);
|
||||||
}
|
}
|
||||||
if (completed) {
|
if (completed) {
|
||||||
future.complete(loc);
|
future.complete(loc);
|
||||||
@ -176,13 +176,13 @@ class AsyncNonMetaRegionLocator {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void removeFromCache(HRegionLocation loc) {
|
private void removeFromCache(HRegionLocation loc) {
|
||||||
TableCache tableCache = cache.get(loc.getRegionInfo().getTable());
|
TableCache tableCache = cache.get(loc.getRegion().getTable());
|
||||||
if (tableCache == null) {
|
if (tableCache == null) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
tableCache.cache.computeIfPresent(loc.getRegionInfo().getStartKey(), (k, oldLoc) -> {
|
tableCache.cache.computeIfPresent(loc.getRegion().getStartKey(), (k, oldLoc) -> {
|
||||||
if (oldLoc.getSeqNum() > loc.getSeqNum() ||
|
if (oldLoc.getSeqNum() > loc.getSeqNum() ||
|
||||||
!oldLoc.getServerName().equals(loc.getServerName())) {
|
!oldLoc.getServerName().equals(loc.getServerName())) {
|
||||||
return oldLoc;
|
return oldLoc;
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
@ -194,16 +194,16 @@ class AsyncNonMetaRegionLocator {
|
|||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Try adding " + loc + " to cache");
|
LOG.trace("Try adding " + loc + " to cache");
|
||||||
}
|
}
|
||||||
byte[] startKey = loc.getRegionInfo().getStartKey();
|
byte[] startKey = loc.getRegion().getStartKey();
|
||||||
HRegionLocation oldLoc = tableCache.cache.putIfAbsent(startKey, loc);
|
HRegionLocation oldLoc = tableCache.cache.putIfAbsent(startKey, loc);
|
||||||
if (oldLoc == null) {
|
if (oldLoc == null) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
if (oldLoc.getSeqNum() > loc.getSeqNum() ||
|
if (oldLoc.getSeqNum() > loc.getSeqNum() ||
|
||||||
oldLoc.getServerName().equals(loc.getServerName())) {
|
oldLoc.getServerName().equals(loc.getServerName())) {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Will not add " + loc + " to cache because the old value " + oldLoc +
|
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;
|
return false;
|
||||||
}
|
}
|
||||||
@ -213,8 +213,8 @@ class AsyncNonMetaRegionLocator {
|
|||||||
}
|
}
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Will not add " + loc + " to cache because the old value " + oldValue +
|
LOG.trace("Will not add " + loc + " to cache because the old value " + oldValue +
|
||||||
" is newer than us or has the same server name." +
|
" is newer than us or has the same server name." +
|
||||||
" Maybe it is updated before we replace it");
|
" Maybe it is updated before we replace it");
|
||||||
}
|
}
|
||||||
return oldValue;
|
return oldValue;
|
||||||
});
|
});
|
||||||
@ -223,7 +223,7 @@ class AsyncNonMetaRegionLocator {
|
|||||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
|
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
|
||||||
justification = "Called by lambda expression")
|
justification = "Called by lambda expression")
|
||||||
private void addToCache(HRegionLocation loc) {
|
private void addToCache(HRegionLocation loc) {
|
||||||
addToCache(getTableCache(loc.getRegionInfo().getTable()), loc);
|
addToCache(getTableCache(loc.getRegion().getTable()), loc);
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Try adding " + loc + " to cache");
|
LOG.trace("Try adding " + loc + " to cache");
|
||||||
}
|
}
|
||||||
@ -232,9 +232,8 @@ class AsyncNonMetaRegionLocator {
|
|||||||
private void complete(TableName tableName, LocateRequest req, HRegionLocation loc,
|
private void complete(TableName tableName, LocateRequest req, HRegionLocation loc,
|
||||||
Throwable error) {
|
Throwable error) {
|
||||||
if (error != null) {
|
if (error != null) {
|
||||||
LOG.warn(
|
LOG.warn("Failed to locate region in '" + tableName + "', row='" +
|
||||||
"Failed to locate region in '" + tableName + "', row='" + Bytes.toStringBinary(req.row)
|
Bytes.toStringBinary(req.row) + "', locateType=" + req.locateType, error);
|
||||||
+ "', locateType=" + req.locateType, error);
|
|
||||||
}
|
}
|
||||||
Optional<LocateRequest> toSend = Optional.empty();
|
Optional<LocateRequest> toSend = Optional.empty();
|
||||||
TableCache tableCache = getTableCache(tableName);
|
TableCache tableCache = getTableCache(tableName);
|
||||||
@ -283,7 +282,7 @@ class AsyncNonMetaRegionLocator {
|
|||||||
RegionLocations locs = MetaTableAccessor.getRegionLocations(results.get(0));
|
RegionLocations locs = MetaTableAccessor.getRegionLocations(results.get(0));
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("The fetched location of '" + tableName + "', row='" +
|
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) {
|
if (locs == null || locs.getDefaultRegionLocation() == null) {
|
||||||
complete(tableName, req, null,
|
complete(tableName, req, null,
|
||||||
@ -292,7 +291,7 @@ class AsyncNonMetaRegionLocator {
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
HRegionLocation loc = locs.getDefaultRegionLocation();
|
HRegionLocation loc = locs.getDefaultRegionLocation();
|
||||||
HRegionInfo info = loc.getRegionInfo();
|
RegionInfo info = loc.getRegion();
|
||||||
if (info == null) {
|
if (info == null) {
|
||||||
complete(tableName, req, null,
|
complete(tableName, req, null,
|
||||||
new IOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
|
new IOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
|
||||||
@ -308,12 +307,12 @@ class AsyncNonMetaRegionLocator {
|
|||||||
complete(tableName, req, null,
|
complete(tableName, req, null,
|
||||||
new RegionOfflineException(
|
new RegionOfflineException(
|
||||||
"the only available region for the required row is a split parent," +
|
"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;
|
return;
|
||||||
}
|
}
|
||||||
if (info.isOffline()) {
|
if (info.isOffline()) {
|
||||||
complete(tableName, req, null, new RegionOfflineException("the region is offline, could" +
|
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;
|
return;
|
||||||
}
|
}
|
||||||
if (loc.getServerName() == null) {
|
if (loc.getServerName() == null) {
|
||||||
@ -332,11 +331,11 @@ class AsyncNonMetaRegionLocator {
|
|||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
HRegionLocation loc = entry.getValue();
|
HRegionLocation loc = entry.getValue();
|
||||||
byte[] endKey = loc.getRegionInfo().getEndKey();
|
byte[] endKey = loc.getRegion().getEndKey();
|
||||||
if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
|
if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
|
LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
|
||||||
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.CURRENT);
|
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.CURRENT);
|
||||||
}
|
}
|
||||||
return loc;
|
return loc;
|
||||||
} else {
|
} else {
|
||||||
@ -347,16 +346,16 @@ class AsyncNonMetaRegionLocator {
|
|||||||
private HRegionLocation locateRowBeforeInCache(TableCache tableCache, TableName tableName,
|
private HRegionLocation locateRowBeforeInCache(TableCache tableCache, TableName tableName,
|
||||||
byte[] row) {
|
byte[] row) {
|
||||||
Map.Entry<byte[], HRegionLocation> entry =
|
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) {
|
if (entry == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
HRegionLocation loc = entry.getValue();
|
HRegionLocation loc = entry.getValue();
|
||||||
if (isEmptyStopRow(loc.getRegionInfo().getEndKey()) ||
|
if (isEmptyStopRow(loc.getRegion().getEndKey()) ||
|
||||||
Bytes.compareTo(loc.getRegionInfo().getEndKey(), row) >= 0) {
|
Bytes.compareTo(loc.getRegion().getEndKey(), row) >= 0) {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
|
LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
|
||||||
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.BEFORE);
|
Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.BEFORE);
|
||||||
}
|
}
|
||||||
return loc;
|
return loc;
|
||||||
} else {
|
} else {
|
||||||
@ -367,7 +366,7 @@ class AsyncNonMetaRegionLocator {
|
|||||||
private void locateInMeta(TableName tableName, LocateRequest req) {
|
private void locateInMeta(TableName tableName, LocateRequest req) {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Try locate '" + tableName + "', row='" + Bytes.toStringBinary(req.row) +
|
LOG.trace("Try locate '" + tableName + "', row='" + Bytes.toStringBinary(req.row) +
|
||||||
"', locateType=" + req.locateType + " in meta");
|
"', locateType=" + req.locateType + " in meta");
|
||||||
}
|
}
|
||||||
byte[] metaKey;
|
byte[] metaKey;
|
||||||
if (req.locateType.equals(RegionLocateType.BEFORE)) {
|
if (req.locateType.equals(RegionLocateType.BEFORE)) {
|
||||||
@ -380,7 +379,7 @@ class AsyncNonMetaRegionLocator {
|
|||||||
} else {
|
} else {
|
||||||
metaKey = createRegionName(tableName, req.row, NINES, false);
|
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)
|
.scanAll(new Scan().withStartRow(metaKey).setReversed(true).addFamily(CATALOG_FAMILY)
|
||||||
.setOneRowLimit())
|
.setOneRowLimit())
|
||||||
.whenComplete((results, error) -> onScanComplete(tableName, req, results, error));
|
.whenComplete((results, error) -> onScanComplete(tableName, req, results, error));
|
||||||
@ -389,8 +388,8 @@ class AsyncNonMetaRegionLocator {
|
|||||||
private HRegionLocation locateInCache(TableCache tableCache, TableName tableName, byte[] row,
|
private HRegionLocation locateInCache(TableCache tableCache, TableName tableName, byte[] row,
|
||||||
RegionLocateType locateType) {
|
RegionLocateType locateType) {
|
||||||
return locateType.equals(RegionLocateType.BEFORE)
|
return locateType.equals(RegionLocateType.BEFORE)
|
||||||
? locateRowBeforeInCache(tableCache, tableName, row)
|
? locateRowBeforeInCache(tableCache, tableName, row)
|
||||||
: locateRowInCache(tableCache, tableName, row);
|
: locateRowInCache(tableCache, tableName, row);
|
||||||
}
|
}
|
||||||
|
|
||||||
// locateToPrevious is true means we will use the start key of a region to locate the region
|
// 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) {
|
void updateCachedLocation(HRegionLocation loc, Throwable exception) {
|
||||||
AsyncRegionLocator.updateCachedLocation(loc, exception, l -> {
|
AsyncRegionLocator.updateCachedLocation(loc, exception, l -> {
|
||||||
TableCache tableCache = cache.get(l.getRegionInfo().getTable());
|
TableCache tableCache = cache.get(l.getRegion().getTable());
|
||||||
if (tableCache == null) {
|
if (tableCache == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
return tableCache.cache.get(l.getRegionInfo().getStartKey());
|
return tableCache.cache.get(l.getRegion().getStartKey());
|
||||||
}, this::addToCache, this::removeFromCache);
|
}, this::addToCache, this::removeFromCache);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -153,7 +153,7 @@ class AsyncRpcRetryingCallerFactory {
|
|||||||
|
|
||||||
private ScanResultCache resultCache;
|
private ScanResultCache resultCache;
|
||||||
|
|
||||||
private RawScanResultConsumer consumer;
|
private AdvancedScanResultConsumer consumer;
|
||||||
|
|
||||||
private ClientService.Interface stub;
|
private ClientService.Interface stub;
|
||||||
|
|
||||||
@ -192,7 +192,7 @@ class AsyncRpcRetryingCallerFactory {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ScanSingleRegionCallerBuilder consumer(RawScanResultConsumer consumer) {
|
public ScanSingleRegionCallerBuilder consumer(AdvancedScanResultConsumer consumer) {
|
||||||
this.consumer = consumer;
|
this.consumer = consumer;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
@ -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.updateResultsMetrics;
|
||||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.updateServerSideMetrics;
|
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.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.CompletableFuture;
|
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.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer.ScanResumer;
|
||||||
import org.apache.hadoop.hbase.client.RawScanResultConsumer.ScanResumer;
|
|
||||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||||
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
||||||
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
|
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
|
||||||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
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.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
|
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.ClientService.Interface;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
|
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.shaded.protobuf.generated.ClientProtos.ScanResponse;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retry caller for scanning a region.
|
* Retry caller for scanning a region.
|
||||||
@ -84,7 +82,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||||||
|
|
||||||
private final ScanResultCache resultCache;
|
private final ScanResultCache resultCache;
|
||||||
|
|
||||||
private final RawScanResultConsumer consumer;
|
private final AdvancedScanResultConsumer consumer;
|
||||||
|
|
||||||
private final ClientService.Interface stub;
|
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
|
// 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
|
// package private methods can only be called within the implementation of
|
||||||
// AsyncScanSingleRegionRpcRetryingCaller.
|
// 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.
|
// Make sure the methods are only called in this thread.
|
||||||
private final Thread callerThread;
|
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
|
// 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
|
// package private methods can only be called within the implementation of
|
||||||
// AsyncScanSingleRegionRpcRetryingCaller.
|
// AsyncScanSingleRegionRpcRetryingCaller.
|
||||||
private final class ScanResumerImpl implements RawScanResultConsumer.ScanResumer {
|
private final class ScanResumerImpl implements AdvancedScanResultConsumer.ScanResumer {
|
||||||
|
|
||||||
// INITIALIZED -> SUSPENDED -> RESUMED
|
// INITIALIZED -> SUSPENDED -> RESUMED
|
||||||
// INITIALIZED -> RESUMED
|
// INITIALIZED -> RESUMED
|
||||||
@ -301,7 +299,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||||||
|
|
||||||
public AsyncScanSingleRegionRpcRetryingCaller(HashedWheelTimer retryTimer,
|
public AsyncScanSingleRegionRpcRetryingCaller(HashedWheelTimer retryTimer,
|
||||||
AsyncConnectionImpl conn, Scan scan, ScanMetrics scanMetrics, long scannerId,
|
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,
|
HRegionLocation loc, boolean isRegionServerRemote, long scannerLeaseTimeoutPeriodNs,
|
||||||
long pauseNs, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) {
|
long pauseNs, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) {
|
||||||
this.retryTimer = retryTimer;
|
this.retryTimer = retryTimer;
|
||||||
@ -344,8 +342,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||||||
stub.scan(controller, req, resp -> {
|
stub.scan(controller, req, resp -> {
|
||||||
if (controller.failed()) {
|
if (controller.failed()) {
|
||||||
LOG.warn("Call to " + loc.getServerName() + " for closing scanner id = " + scannerId +
|
LOG.warn("Call to " + loc.getServerName() + " for closing scanner id = " + scannerId +
|
||||||
" for " + loc.getRegionInfo().getEncodedName() + " of " +
|
" for " + loc.getRegion().getEncodedName() + " of " +
|
||||||
loc.getRegionInfo().getTable() + " failed, ignore, probably already closed",
|
loc.getRegion().getTable() + " failed, ignore, probably already closed",
|
||||||
controller.getFailed());
|
controller.getFailed());
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
@ -384,7 +382,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||||||
error = translateException(error);
|
error = translateException(error);
|
||||||
if (tries > startLogErrorsCnt) {
|
if (tries > startLogErrorsCnt) {
|
||||||
LOG.warn("Call to " + loc.getServerName() + " for scanner id = " + scannerId + " for " +
|
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 = " +
|
" failed, , tries = " + tries + ", maxAttempts = " + maxAttempts + ", timeout = " +
|
||||||
TimeUnit.NANOSECONDS.toMillis(scanTimeoutNs) + " ms, time elapsed = " + elapsedMs() +
|
TimeUnit.NANOSECONDS.toMillis(scanTimeoutNs) + " ms, time elapsed = " + elapsedMs() +
|
||||||
" ms",
|
" ms",
|
||||||
@ -433,18 +431,18 @@ class AsyncScanSingleRegionRpcRetryingCaller {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void completeWhenNoMoreResultsInRegion() {
|
private void completeWhenNoMoreResultsInRegion() {
|
||||||
if (noMoreResultsForScan(scan, loc.getRegionInfo())) {
|
if (noMoreResultsForScan(scan, loc.getRegion())) {
|
||||||
completeNoMoreResults();
|
completeNoMoreResults();
|
||||||
} else {
|
} else {
|
||||||
completeWithNextStartRow(loc.getRegionInfo().getEndKey(), true);
|
completeWithNextStartRow(loc.getRegion().getEndKey(), true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void completeReversedWhenNoMoreResultsInRegion() {
|
private void completeReversedWhenNoMoreResultsInRegion() {
|
||||||
if (noMoreResultsForReverseScan(scan, loc.getRegionInfo())) {
|
if (noMoreResultsForReverseScan(scan, loc.getRegion())) {
|
||||||
completeNoMoreResults();
|
completeNoMoreResults();
|
||||||
} else {
|
} else {
|
||||||
completeWithNextStartRow(loc.getRegionInfo().getStartKey(), false);
|
completeWithNextStartRow(loc.getRegion().getStartKey(), false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -68,7 +68,7 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
|
|||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
onError(e,
|
onError(e,
|
||||||
() -> "Get async stub to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row)
|
() -> "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));
|
err -> conn.getLocator().updateCachedLocation(loc, err));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
@ -78,7 +78,7 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
|
|||||||
if (error != null) {
|
if (error != null) {
|
||||||
onError(error,
|
onError(error,
|
||||||
() -> "Call to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) + "' in "
|
() -> "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));
|
err -> conn.getLocator().updateCachedLocation(loc, err));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -17,20 +17,277 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
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.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>
|
* <p>
|
||||||
* The implementation is required to be thread safe.
|
* The implementation is required to be thread safe.
|
||||||
* <p>
|
* <p>
|
||||||
* The implementation should make sure that user can do everything they want to the returned
|
* Usually the implementation will not throw any exception directly. You need to get the exception
|
||||||
* {@code CompletableFuture} without breaking anything. Usually the implementation will require user
|
* from the returned {@link CompletableFuture}.
|
||||||
* to provide a {@code ExecutorService}.
|
|
||||||
* @since 2.0.0
|
* @since 2.0.0
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Public
|
@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.
|
* Gets a scanner on the current table for the given family.
|
||||||
@ -59,13 +316,300 @@ public interface AsyncTable extends AsyncTableBase {
|
|||||||
ResultScanner getScanner(Scan scan);
|
ResultScanner getScanner(Scan scan);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The scan API uses the observer pattern. All results that match the given scan object will be
|
* Return all the results that match the given scan object.
|
||||||
* passed to the given {@code consumer} by calling {@link ScanResultConsumer#onNext(Result)}.
|
* <p>
|
||||||
* {@link ScanResultConsumer#onComplete()} means the scan is finished, and
|
* Notice that usually you should use this method with a {@link Scan} object that has limit set.
|
||||||
* {@link ScanResultConsumer#onError(Throwable)} means we hit an unrecoverable error and the scan
|
* For example, if you want to get the closest row after a given row, you could do this:
|
||||||
* is terminated.
|
* <p>
|
||||||
* @param scan A configured {@link Scan} object.
|
*
|
||||||
* @param consumer the consumer used to receive results.
|
* <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);
|
||||||
}
|
}
|
||||||
|
@ -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));
|
|
||||||
}
|
|
||||||
}
|
|
@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
|
|||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* For creating {@link AsyncTable} or {@link RawAsyncTable}.
|
* For creating {@link AsyncTable}.
|
||||||
* <p>
|
* <p>
|
||||||
* The implementation should have default configurations set before returning the builder to user.
|
* 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
|
* 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
|
* @since 2.0.0
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Public
|
@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
|
* 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 #setMaxRetries(int)
|
||||||
* @see #setScanTimeout(long, TimeUnit)
|
* @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
|
* 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.
|
* operation in a scan, such as openScanner or next.
|
||||||
* @see #setScanTimeout(long, TimeUnit)
|
* @see #setScanTimeout(long, TimeUnit)
|
||||||
*/
|
*/
|
||||||
AsyncTableBuilder<T> setScanTimeout(long timeout, TimeUnit unit);
|
AsyncTableBuilder<C> setScanTimeout(long timeout, TimeUnit unit);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set timeout for each rpc request.
|
* 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
|
* Notice that this will <strong>NOT</strong> change the rpc timeout for read(get, scan) request
|
||||||
* and write request(put, delete).
|
* 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.
|
* 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.
|
* 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
|
* Set the base pause time for retrying. We use an exponential policy to generate sleep time when
|
||||||
* retrying.
|
* 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.
|
* 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 #setMaxAttempts(int)
|
||||||
* @see #setOperationTimeout(long, TimeUnit)
|
* @see #setOperationTimeout(long, TimeUnit)
|
||||||
*/
|
*/
|
||||||
default AsyncTableBuilder<T> setMaxRetries(int maxRetries) {
|
default AsyncTableBuilder<C> setMaxRetries(int maxRetries) {
|
||||||
return setMaxAttempts(retries2Attempts(maxRetries));
|
return setMaxAttempts(retries2Attempts(maxRetries));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -98,15 +98,15 @@ public interface AsyncTableBuilder<T extends AsyncTableBase> {
|
|||||||
* @see #setMaxRetries(int)
|
* @see #setMaxRetries(int)
|
||||||
* @see #setOperationTimeout(long, TimeUnit)
|
* @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.
|
* 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();
|
||||||
}
|
}
|
||||||
|
@ -28,7 +28,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||||||
* Base class for all asynchronous table builders.
|
* Base class for all asynchronous table builders.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
abstract class AsyncTableBuilderBase<T extends AsyncTableBase> implements AsyncTableBuilder<T> {
|
abstract class AsyncTableBuilderBase<C extends ScanResultConsumerBase>
|
||||||
|
implements AsyncTableBuilder<C> {
|
||||||
|
|
||||||
protected TableName tableName;
|
protected TableName tableName;
|
||||||
|
|
||||||
@ -51,7 +52,7 @@ abstract class AsyncTableBuilderBase<T extends AsyncTableBase> implements AsyncT
|
|||||||
AsyncTableBuilderBase(TableName tableName, AsyncConnectionConfiguration connConf) {
|
AsyncTableBuilderBase(TableName tableName, AsyncConnectionConfiguration connConf) {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
this.operationTimeoutNs = tableName.isSystemTable() ? connConf.getMetaOperationTimeoutNs()
|
this.operationTimeoutNs = tableName.isSystemTable() ? connConf.getMetaOperationTimeoutNs()
|
||||||
: connConf.getOperationTimeoutNs();
|
: connConf.getOperationTimeoutNs();
|
||||||
this.scanTimeoutNs = connConf.getScanTimeoutNs();
|
this.scanTimeoutNs = connConf.getScanTimeoutNs();
|
||||||
this.rpcTimeoutNs = connConf.getRpcTimeoutNs();
|
this.rpcTimeoutNs = connConf.getRpcTimeoutNs();
|
||||||
this.readRpcTimeoutNs = connConf.getReadRpcTimeoutNs();
|
this.readRpcTimeoutNs = connConf.getReadRpcTimeoutNs();
|
||||||
@ -62,49 +63,49 @@ abstract class AsyncTableBuilderBase<T extends AsyncTableBase> implements AsyncT
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setOperationTimeout(long timeout, TimeUnit unit) {
|
public AsyncTableBuilderBase<C> setOperationTimeout(long timeout, TimeUnit unit) {
|
||||||
this.operationTimeoutNs = unit.toNanos(timeout);
|
this.operationTimeoutNs = unit.toNanos(timeout);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setScanTimeout(long timeout, TimeUnit unit) {
|
public AsyncTableBuilderBase<C> setScanTimeout(long timeout, TimeUnit unit) {
|
||||||
this.scanTimeoutNs = unit.toNanos(timeout);
|
this.scanTimeoutNs = unit.toNanos(timeout);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setRpcTimeout(long timeout, TimeUnit unit) {
|
public AsyncTableBuilderBase<C> setRpcTimeout(long timeout, TimeUnit unit) {
|
||||||
this.rpcTimeoutNs = unit.toNanos(timeout);
|
this.rpcTimeoutNs = unit.toNanos(timeout);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setReadRpcTimeout(long timeout, TimeUnit unit) {
|
public AsyncTableBuilderBase<C> setReadRpcTimeout(long timeout, TimeUnit unit) {
|
||||||
this.readRpcTimeoutNs = unit.toNanos(timeout);
|
this.readRpcTimeoutNs = unit.toNanos(timeout);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setWriteRpcTimeout(long timeout, TimeUnit unit) {
|
public AsyncTableBuilderBase<C> setWriteRpcTimeout(long timeout, TimeUnit unit) {
|
||||||
this.writeRpcTimeoutNs = unit.toNanos(timeout);
|
this.writeRpcTimeoutNs = unit.toNanos(timeout);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setRetryPause(long pause, TimeUnit unit) {
|
public AsyncTableBuilderBase<C> setRetryPause(long pause, TimeUnit unit) {
|
||||||
this.pauseNs = unit.toNanos(pause);
|
this.pauseNs = unit.toNanos(pause);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setMaxAttempts(int maxAttempts) {
|
public AsyncTableBuilderBase<C> setMaxAttempts(int maxAttempts) {
|
||||||
this.maxAttempts = maxAttempts;
|
this.maxAttempts = maxAttempts;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AsyncTableBuilderBase<T> setStartLogErrorsCnt(int startLogErrorsCnt) {
|
public AsyncTableBuilderBase<C> setStartLogErrorsCnt(int startLogErrorsCnt) {
|
||||||
this.startLogErrorsCnt = startLogErrorsCnt;
|
this.startLogErrorsCnt = startLogErrorsCnt;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
@ -19,34 +19,37 @@ package org.apache.hadoop.hbase.client;
|
|||||||
|
|
||||||
import static java.util.stream.Collectors.toList;
|
import static java.util.stream.Collectors.toList;
|
||||||
|
|
||||||
|
import com.google.protobuf.RpcChannel;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.CompareOperator;
|
import org.apache.hadoop.hbase.CompareOperator;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
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
|
@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 ExecutorService pool;
|
||||||
|
|
||||||
private final long defaultScannerMaxResultSize;
|
AsyncTableImpl(AsyncConnectionImpl conn, AsyncTable<AdvancedScanResultConsumer> rawTable,
|
||||||
|
ExecutorService pool) {
|
||||||
AsyncTableImpl(AsyncConnectionImpl conn, RawAsyncTable rawTable, ExecutorService pool) {
|
|
||||||
this.rawTable = rawTable;
|
this.rawTable = rawTable;
|
||||||
this.pool = pool;
|
this.pool = pool;
|
||||||
this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -172,16 +175,9 @@ class AsyncTableImpl implements AsyncTable {
|
|||||||
return wrap(rawTable.scanAll(scan));
|
return wrap(rawTable.scanAll(scan));
|
||||||
}
|
}
|
||||||
|
|
||||||
private long resultSize2CacheSize(long maxResultSize) {
|
|
||||||
// * 2 if possible
|
|
||||||
return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ResultScanner getScanner(Scan scan) {
|
public ResultScanner getScanner(Scan scan) {
|
||||||
return new AsyncTableResultScanner(rawTable, ReflectionUtils.newInstance(scan.getClass(), scan),
|
return rawTable.getScanner(scan);
|
||||||
resultSize2CacheSize(
|
|
||||||
scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void scan0(Scan scan, ScanResultConsumer consumer) {
|
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) {
|
public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {
|
||||||
return rawTable.<T> batch(actions).stream().map(this::wrap).collect(toList());
|
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();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -37,11 +37,11 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
|
|||||||
* {@code 2 * scan.getMaxResultSize()}.
|
* {@code 2 * scan.getMaxResultSize()}.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
|
class AsyncTableResultScanner implements ResultScanner, AdvancedScanResultConsumer {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(AsyncTableResultScanner.class);
|
private static final Log LOG = LogFactory.getLog(AsyncTableResultScanner.class);
|
||||||
|
|
||||||
private final RawAsyncTable rawTable;
|
private final AsyncTable<AdvancedScanResultConsumer> rawTable;
|
||||||
|
|
||||||
private final long maxCacheSize;
|
private final long maxCacheSize;
|
||||||
|
|
||||||
@ -59,7 +59,8 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
|
|||||||
|
|
||||||
private ScanResumer resumer;
|
private ScanResumer resumer;
|
||||||
|
|
||||||
public AsyncTableResultScanner(RawAsyncTable table, Scan scan, long maxCacheSize) {
|
public AsyncTableResultScanner(AsyncTable<AdvancedScanResultConsumer> table, Scan scan,
|
||||||
|
long maxCacheSize) {
|
||||||
this.rawTable = table;
|
this.rawTable = table;
|
||||||
this.maxCacheSize = maxCacheSize;
|
this.maxCacheSize = maxCacheSize;
|
||||||
this.scan = scan;
|
this.scan = scan;
|
||||||
@ -74,8 +75,8 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
|
|||||||
private void stopPrefetch(ScanController controller) {
|
private void stopPrefetch(ScanController controller) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug(String.format("0x%x", System.identityHashCode(this)) +
|
LOG.debug(String.format("0x%x", System.identityHashCode(this)) +
|
||||||
" stop prefetching when scanning " + rawTable.getName() + " as the cache size " +
|
" stop prefetching when scanning " + rawTable.getName() + " as the cache size " +
|
||||||
cacheSize + " is greater than the maxCacheSize " + maxCacheSize);
|
cacheSize + " is greater than the maxCacheSize " + maxCacheSize);
|
||||||
}
|
}
|
||||||
resumer = controller.suspend();
|
resumer = controller.suspend();
|
||||||
}
|
}
|
||||||
|
@ -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_END_ROW;
|
||||||
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_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.io.IOException;
|
||||||
import java.lang.reflect.UndeclaredThrowableException;
|
import java.lang.reflect.UndeclaredThrowableException;
|
||||||
import java.net.InetAddress;
|
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.Cell;
|
||||||
import org.apache.hadoop.hbase.CellComparator;
|
import org.apache.hadoop.hbase.CellComparator;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
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.MasterNotRunningException;
|
||||||
|
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
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.client.metrics.ScanMetrics;
|
||||||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.security.UserProvider;
|
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.com.google.protobuf.ServiceException;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
|
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.AdminProtos.AdminService;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
|
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.ClientProtos.ScanResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
|
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.
|
* 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())) {
|
if (isEmptyStopRow(info.getEndKey())) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -392,7 +391,7 @@ public final class ConnectionUtils {
|
|||||||
return c > 0 || (c == 0 && !scan.includeStopRow());
|
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())) {
|
if (isEmptyStartRow(info.getStartKey())) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -19,6 +19,11 @@ package org.apache.hadoop.hbase.client;
|
|||||||
|
|
||||||
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
|
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.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
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.AdminRequestCallerBuilder;
|
||||||
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
|
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
|
||||||
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder;
|
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.Scan.ReadType;
|
||||||
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
|
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
|
||||||
import org.apache.hadoop.hbase.client.replication.TableCFs;
|
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.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
|
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.HashedWheelTimer;
|
||||||
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
|
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.ReplicationProtos.UpdateReplicationPeerConfigResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
|
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.
|
* The implementation of AsyncAdmin.
|
||||||
* <p>
|
* <p>
|
||||||
@ -263,7 +263,7 @@ import com.google.protobuf.RpcChannel;
|
|||||||
* @see AsyncConnection#getAdminBuilder()
|
* @see AsyncConnection#getAdminBuilder()
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class RawAsyncHBaseAdmin implements AsyncAdmin {
|
class RawAsyncHBaseAdmin implements AsyncAdmin {
|
||||||
public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE = "flush-table-proc";
|
public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE = "flush-table-proc";
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(AsyncHBaseAdmin.class);
|
private static final Log LOG = LogFactory.getLog(AsyncHBaseAdmin.class);
|
||||||
@ -272,7 +272,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||||||
|
|
||||||
private final HashedWheelTimer retryTimer;
|
private final HashedWheelTimer retryTimer;
|
||||||
|
|
||||||
private final RawAsyncTable metaTable;
|
private final AsyncTable<AdvancedScanResultConsumer> metaTable;
|
||||||
|
|
||||||
private final long rpcTimeoutNs;
|
private final long rpcTimeoutNs;
|
||||||
|
|
||||||
@ -290,7 +290,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||||||
AsyncAdminBuilderBase builder) {
|
AsyncAdminBuilderBase builder) {
|
||||||
this.connection = connection;
|
this.connection = connection;
|
||||||
this.retryTimer = retryTimer;
|
this.retryTimer = retryTimer;
|
||||||
this.metaTable = connection.getRawTable(META_TABLE_NAME);
|
this.metaTable = connection.getTable(META_TABLE_NAME);
|
||||||
this.rpcTimeoutNs = builder.rpcTimeoutNs;
|
this.rpcTimeoutNs = builder.rpcTimeoutNs;
|
||||||
this.operationTimeoutNs = builder.operationTimeoutNs;
|
this.operationTimeoutNs = builder.operationTimeoutNs;
|
||||||
this.pauseNs = builder.pauseNs;
|
this.pauseNs = builder.pauseNs;
|
||||||
@ -1442,8 +1442,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||||||
public CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter) {
|
public CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter) {
|
||||||
CompletableFuture<List<QuotaSettings>> future = new CompletableFuture<>();
|
CompletableFuture<List<QuotaSettings>> future = new CompletableFuture<>();
|
||||||
Scan scan = QuotaTableUtil.makeScan(filter);
|
Scan scan = QuotaTableUtil.makeScan(filter);
|
||||||
this.connection.getRawTableBuilder(QuotaTableUtil.QUOTA_TABLE_NAME).build()
|
this.connection.getTableBuilder(QuotaTableUtil.QUOTA_TABLE_NAME).build()
|
||||||
.scan(scan, new RawScanResultConsumer() {
|
.scan(scan, new AdvancedScanResultConsumer() {
|
||||||
List<QuotaSettings> settings = new ArrayList<>();
|
List<QuotaSettings> settings = new ArrayList<>();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -3001,7 +3001,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
||||||
CoprocessorCallable<S, R> callable) {
|
ServiceCaller<S, R> callable) {
|
||||||
MasterCoprocessorRpcChannelImpl channel =
|
MasterCoprocessorRpcChannelImpl channel =
|
||||||
new MasterCoprocessorRpcChannelImpl(this.<Message> newMasterCaller());
|
new MasterCoprocessorRpcChannelImpl(this.<Message> newMasterCaller());
|
||||||
S stub = stubMaker.apply(channel);
|
S stub = stubMaker.apply(channel);
|
||||||
@ -3019,7 +3019,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
||||||
CoprocessorCallable<S, R> callable, ServerName serverName) {
|
ServiceCaller<S, R> callable, ServerName serverName) {
|
||||||
RegionServerCoprocessorRpcChannelImpl channel =
|
RegionServerCoprocessorRpcChannelImpl channel =
|
||||||
new RegionServerCoprocessorRpcChannelImpl(this.<Message> newServerCaller().serverName(
|
new RegionServerCoprocessorRpcChannelImpl(this.<Message> newServerCaller().serverName(
|
||||||
serverName));
|
serverName));
|
||||||
|
@ -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);
|
|
||||||
}
|
|
@ -62,9 +62,16 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* The implementation of RawAsyncTable.
|
* 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
|
@InterfaceAudience.Private
|
||||||
class RawAsyncTableImpl implements RawAsyncTable {
|
class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
|
||||||
|
|
||||||
private final AsyncConnectionImpl conn;
|
private final AsyncConnectionImpl conn;
|
||||||
|
|
||||||
@ -102,7 +109,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
this.maxAttempts = builder.maxAttempts;
|
this.maxAttempts = builder.maxAttempts;
|
||||||
this.startLogErrorsCnt = builder.startLogErrorsCnt;
|
this.startLogErrorsCnt = builder.startLogErrorsCnt;
|
||||||
this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching()
|
this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching()
|
||||||
: conn.connConf.getScannerCaching();
|
: conn.connConf.getScannerCaching();
|
||||||
this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();
|
this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -270,7 +277,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
@Override
|
@Override
|
||||||
public CheckAndMutateBuilder qualifier(byte[] qualifier) {
|
public CheckAndMutateBuilder qualifier(byte[] qualifier) {
|
||||||
this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" +
|
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;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -290,7 +297,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
|
|
||||||
private void preCheck() {
|
private void preCheck() {
|
||||||
Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" +
|
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
|
@Override
|
||||||
@ -354,14 +361,12 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
} else {
|
} else {
|
||||||
try {
|
try {
|
||||||
org.apache.hadoop.hbase.client.MultiResponse multiResp =
|
org.apache.hadoop.hbase.client.MultiResponse multiResp =
|
||||||
ResponseConverter.getResults(req, resp, controller.cellScanner());
|
ResponseConverter.getResults(req, resp, controller.cellScanner());
|
||||||
Throwable ex = multiResp.getException(regionName);
|
Throwable ex = multiResp.getException(regionName);
|
||||||
if (ex != null) {
|
if (ex != null) {
|
||||||
future
|
future.completeExceptionally(ex instanceof IOException ? ex
|
||||||
.completeExceptionally(ex instanceof IOException ? ex
|
: new IOException(
|
||||||
: new IOException(
|
"Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
|
||||||
"Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()),
|
|
||||||
ex));
|
|
||||||
} else {
|
} else {
|
||||||
future.complete(respConverter
|
future.complete(respConverter
|
||||||
.apply((Result) multiResp.getResults().get(regionName).result.get(0)));
|
.apply((Result) multiResp.getResults().get(regionName).result.get(0)));
|
||||||
@ -400,11 +405,28 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
return newScan;
|
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
|
@Override
|
||||||
public CompletableFuture<List<Result>> scanAll(Scan scan) {
|
public CompletableFuture<List<Result>> scanAll(Scan scan) {
|
||||||
CompletableFuture<List<Result>> future = new CompletableFuture<>();
|
CompletableFuture<List<Result>> future = new CompletableFuture<>();
|
||||||
List<Result> scanResults = new ArrayList<>();
|
List<Result> scanResults = new ArrayList<>();
|
||||||
scan(scan, new RawScanResultConsumer() {
|
scan(scan, new AdvancedScanResultConsumer() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onNext(Result[] results, ScanController controller) {
|
public void onNext(Result[] results, ScanController controller) {
|
||||||
@ -424,11 +446,6 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
return future;
|
return future;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void scan(Scan scan, RawScanResultConsumer consumer) {
|
|
||||||
new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, pauseNs,
|
|
||||||
maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<CompletableFuture<Result>> get(List<Get> gets) {
|
public List<CompletableFuture<Result>> get(List<Get> gets) {
|
||||||
return batch(gets, readRpcTimeoutNs);
|
return batch(gets, readRpcTimeoutNs);
|
||||||
@ -487,7 +504,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
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,
|
RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
|
||||||
region, row, rpcTimeoutNs, operationTimeoutNs);
|
region, row, rpcTimeoutNs, operationTimeoutNs);
|
||||||
S stub = stubMaker.apply(channel);
|
S stub = stubMaker.apply(channel);
|
||||||
@ -505,7 +522,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
|
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);
|
return coprocessorService(stubMaker, callable, null, row);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -527,7 +544,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
|
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,
|
List<HRegionLocation> locs, byte[] endKey, boolean endKeyInclusive,
|
||||||
AtomicBoolean locateFinished, AtomicInteger unfinishedRequest, HRegionLocation loc,
|
AtomicBoolean locateFinished, AtomicInteger unfinishedRequest, HRegionLocation loc,
|
||||||
Throwable error) {
|
Throwable error) {
|
||||||
@ -563,7 +580,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
|
|
||||||
private final Function<RpcChannel, S> stubMaker;
|
private final Function<RpcChannel, S> stubMaker;
|
||||||
|
|
||||||
private final CoprocessorCallable<S, R> callable;
|
private final ServiceCaller<S, R> callable;
|
||||||
|
|
||||||
private final CoprocessorCallback<R> callback;
|
private final CoprocessorCallback<R> callback;
|
||||||
|
|
||||||
@ -576,7 +593,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
private boolean endKeyInclusive;
|
private boolean endKeyInclusive;
|
||||||
|
|
||||||
public CoprocessorServiceBuilderImpl(Function<RpcChannel, S> stubMaker,
|
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.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null");
|
||||||
this.callable = Preconditions.checkNotNull(callable, "callable is null");
|
this.callable = Preconditions.checkNotNull(callable, "callable is null");
|
||||||
this.callback = Preconditions.checkNotNull(callback, "callback 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) {
|
public CoprocessorServiceBuilderImpl<S, R> fromRow(byte[] startKey, boolean inclusive) {
|
||||||
this.startKey = Preconditions.checkNotNull(startKey,
|
this.startKey = Preconditions.checkNotNull(startKey,
|
||||||
"startKey is null. Consider using" +
|
"startKey is null. Consider using" +
|
||||||
" an empty byte array, or just do not call this method if you want to start selection" +
|
" an empty byte array, or just do not call this method if you want to start selection" +
|
||||||
" from the first region");
|
" from the first region");
|
||||||
this.startKeyInclusive = inclusive;
|
this.startKeyInclusive = inclusive;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
@ -596,8 +613,8 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
public CoprocessorServiceBuilderImpl<S, R> toRow(byte[] endKey, boolean inclusive) {
|
public CoprocessorServiceBuilderImpl<S, R> toRow(byte[] endKey, boolean inclusive) {
|
||||||
this.endKey = Preconditions.checkNotNull(endKey,
|
this.endKey = Preconditions.checkNotNull(endKey,
|
||||||
"endKey is null. Consider using" +
|
"endKey is null. Consider using" +
|
||||||
" an empty byte array, or just do not call this method if you want to continue" +
|
" an empty byte array, or just do not call this method if you want to continue" +
|
||||||
" selection to the last region");
|
" selection to the last region");
|
||||||
this.endKeyInclusive = inclusive;
|
this.endKeyInclusive = inclusive;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
@ -614,7 +631,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
|
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) {
|
CoprocessorCallback<R> callback) {
|
||||||
return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
|
return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
|
||||||
}
|
}
|
||||||
|
@ -18,38 +18,20 @@
|
|||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Receives {@link Result} for an asynchronous scan.
|
* 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
|
@InterfaceAudience.Public
|
||||||
public interface ScanResultConsumer {
|
public interface ScanResultConsumer extends ScanResultConsumerBase {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param result the data fetched from HBase service.
|
* @param result the data fetched from HBase service.
|
||||||
* @return {@code false} if you want to terminate the scan process. Otherwise {@code true}
|
* @return {@code false} if you want to terminate the scan process. Otherwise {@code true}
|
||||||
*/
|
*/
|
||||||
boolean onNext(Result result);
|
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) {
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -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) {
|
||||||
|
}
|
||||||
|
}
|
@ -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);
|
||||||
|
}
|
@ -32,9 +32,9 @@ import java.util.concurrent.CompletableFuture;
|
|||||||
|
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.client.RawAsyncTable;
|
import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
|
||||||
import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallback;
|
import org.apache.hadoop.hbase.client.AsyncTable;
|
||||||
import org.apache.hadoop.hbase.client.RawScanResultConsumer;
|
import org.apache.hadoop.hbase.client.AsyncTable.CoprocessorCallback;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
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>
|
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<>();
|
CompletableFuture<R> future = new CompletableFuture<>();
|
||||||
AggregateRequest req;
|
AggregateRequest req;
|
||||||
try {
|
try {
|
||||||
@ -163,7 +163,7 @@ public class AsyncAggregationClient {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static <R, S, P extends Message, Q extends Message, T extends Message> CompletableFuture<R>
|
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<>();
|
CompletableFuture<R> future = new CompletableFuture<>();
|
||||||
AggregateRequest req;
|
AggregateRequest req;
|
||||||
try {
|
try {
|
||||||
@ -201,7 +201,7 @@ public class AsyncAggregationClient {
|
|||||||
|
|
||||||
public static <R, S, P extends Message, Q extends Message, T extends Message>
|
public static <R, S, P extends Message, Q extends Message, T extends Message>
|
||||||
CompletableFuture<Long>
|
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<>();
|
CompletableFuture<Long> future = new CompletableFuture<>();
|
||||||
AggregateRequest req;
|
AggregateRequest req;
|
||||||
try {
|
try {
|
||||||
@ -233,7 +233,7 @@ public class AsyncAggregationClient {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static <R, S, P extends Message, Q extends Message, T extends Message> CompletableFuture<S>
|
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<>();
|
CompletableFuture<S> future = new CompletableFuture<>();
|
||||||
AggregateRequest req;
|
AggregateRequest req;
|
||||||
try {
|
try {
|
||||||
@ -269,7 +269,7 @@ public class AsyncAggregationClient {
|
|||||||
|
|
||||||
public static <R, S, P extends Message, Q extends Message, T extends Message>
|
public static <R, S, P extends Message, Q extends Message, T extends Message>
|
||||||
CompletableFuture<Double>
|
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<>();
|
CompletableFuture<Double> future = new CompletableFuture<>();
|
||||||
AggregateRequest req;
|
AggregateRequest req;
|
||||||
try {
|
try {
|
||||||
@ -307,7 +307,7 @@ public class AsyncAggregationClient {
|
|||||||
|
|
||||||
public static <R, S, P extends Message, Q extends Message, T extends Message>
|
public static <R, S, P extends Message, Q extends Message, T extends Message>
|
||||||
CompletableFuture<Double>
|
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<>();
|
CompletableFuture<Double> future = new CompletableFuture<>();
|
||||||
AggregateRequest req;
|
AggregateRequest req;
|
||||||
try {
|
try {
|
||||||
@ -351,7 +351,7 @@ public class AsyncAggregationClient {
|
|||||||
// the map key is the startRow of the region
|
// the map key is the startRow of the region
|
||||||
private static <R, S, P extends Message, Q extends Message, T extends Message>
|
private static <R, S, P extends Message, Q extends Message, T extends Message>
|
||||||
CompletableFuture<NavigableMap<byte[], S>>
|
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 =
|
CompletableFuture<NavigableMap<byte[], S>> future =
|
||||||
new CompletableFuture<NavigableMap<byte[], S>>();
|
new CompletableFuture<NavigableMap<byte[], S>>();
|
||||||
AggregateRequest req;
|
AggregateRequest req;
|
||||||
@ -388,8 +388,8 @@ public class AsyncAggregationClient {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static <R, S, P extends Message, Q extends Message, T extends Message> void findMedian(
|
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,
|
CompletableFuture<R> future, AsyncTable<AdvancedScanResultConsumer> table,
|
||||||
Scan scan, NavigableMap<byte[], S> sumByRegion) {
|
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);
|
double halfSum = ci.divideForAvg(sumByRegion.values().stream().reduce(ci::add).get(), 2L);
|
||||||
S movingSum = null;
|
S movingSum = null;
|
||||||
byte[] startRow = null;
|
byte[] startRow = null;
|
||||||
@ -410,7 +410,7 @@ public class AsyncAggregationClient {
|
|||||||
NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(family);
|
NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(family);
|
||||||
byte[] weightQualifier = qualifiers.last();
|
byte[] weightQualifier = qualifiers.last();
|
||||||
byte[] valueQualifier = qualifiers.first();
|
byte[] valueQualifier = qualifiers.first();
|
||||||
table.scan(scan, new RawScanResultConsumer() {
|
table.scan(scan, new AdvancedScanResultConsumer() {
|
||||||
|
|
||||||
private S sum = baseSum;
|
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>
|
public static <R, S, P extends Message, Q extends Message, T extends Message>
|
||||||
median(RawAsyncTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
|
CompletableFuture<R> median(AsyncTable<AdvancedScanResultConsumer> table,
|
||||||
|
ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) {
|
||||||
CompletableFuture<R> future = new CompletableFuture<>();
|
CompletableFuture<R> future = new CompletableFuture<>();
|
||||||
sumByRegion(table, ci, scan).whenComplete((sumByRegion, error) -> {
|
sumByRegion(table, ci, scan).whenComplete((sumByRegion, error) -> {
|
||||||
if (error != null) {
|
if (error != null) {
|
||||||
|
@ -55,7 +55,7 @@ public class TestAsyncAggregationClient {
|
|||||||
|
|
||||||
private static AsyncConnection CONN;
|
private static AsyncConnection CONN;
|
||||||
|
|
||||||
private static RawAsyncTable TABLE;
|
private static AsyncTable<AdvancedScanResultConsumer> TABLE;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUp() throws Exception {
|
public static void setUp() throws Exception {
|
||||||
@ -69,7 +69,7 @@ public class TestAsyncAggregationClient {
|
|||||||
}
|
}
|
||||||
UTIL.createTable(TABLE_NAME, CF, splitKeys);
|
UTIL.createTable(TABLE_NAME, CF, splitKeys);
|
||||||
CONN = ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get();
|
CONN = ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get();
|
||||||
TABLE = CONN.getRawTable(TABLE_NAME);
|
TABLE = CONN.getTable(TABLE_NAME);
|
||||||
TABLE.putAll(LongStream.range(0, COUNT)
|
TABLE.putAll(LongStream.range(0, COUNT)
|
||||||
.mapToObj(l -> new Put(Bytes.toBytes(String.format("%03d", l)))
|
.mapToObj(l -> new Put(Bytes.toBytes(String.format("%03d", l)))
|
||||||
.addColumn(CF, CQ, Bytes.toBytes(l)).addColumn(CF, CQ2, Bytes.toBytes(l * l)))
|
.addColumn(CF, CQ, Bytes.toBytes(l)).addColumn(CF, CQ2, Bytes.toBytes(l * l)))
|
||||||
|
@ -141,7 +141,7 @@ public class AsyncClientExample extends Configured implements Tool {
|
|||||||
latch.countDown();
|
latch.countDown();
|
||||||
return;
|
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)))
|
table.put(new Put(getKey(i)).addColumn(FAMILY, QUAL, Bytes.toBytes(i)))
|
||||||
.whenComplete((putResp, putErr) -> {
|
.whenComplete((putResp, putErr) -> {
|
||||||
if (putErr != null) {
|
if (putErr != null) {
|
||||||
|
@ -17,9 +17,24 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client.example;
|
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.Preconditions;
|
||||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
|
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.bootstrap.ServerBootstrap;
|
||||||
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
|
import org.apache.hadoop.hbase.shaded.io.netty.buffer.ByteBuf;
|
||||||
import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
|
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.handler.codec.http.QueryStringDecoder;
|
||||||
import org.apache.hadoop.hbase.shaded.io.netty.util.concurrent.GlobalEventExecutor;
|
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
|
* A simple example on how to use {@link org.apache.hadoop.hbase.client.AsyncTable} to write a fully
|
||||||
* server. The {@link AsyncConnection} will share the same event loop with the HTTP server.
|
* asynchronous HTTP proxy server. The {@link AsyncConnection} will share the same event loop with
|
||||||
|
* the HTTP server.
|
||||||
* <p>
|
* <p>
|
||||||
* The request URL is:
|
* The request URL is:
|
||||||
*
|
*
|
||||||
@ -160,7 +159,7 @@ public class HttpProxyExample {
|
|||||||
|
|
||||||
private void get(ChannelHandlerContext ctx, FullHttpRequest req) {
|
private void get(ChannelHandlerContext ctx, FullHttpRequest req) {
|
||||||
Params params = parse(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)))
|
.addColumn(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier)))
|
||||||
.whenComplete((r, e) -> {
|
.whenComplete((r, e) -> {
|
||||||
if (e != null) {
|
if (e != null) {
|
||||||
@ -181,7 +180,7 @@ public class HttpProxyExample {
|
|||||||
Params params = parse(req);
|
Params params = parse(req);
|
||||||
byte[] value = new byte[req.content().readableBytes()];
|
byte[] value = new byte[req.content().readableBytes()];
|
||||||
req.content().readBytes(value);
|
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))
|
.addColumn(Bytes.toBytes(params.family), Bytes.toBytes(params.qualifier), value))
|
||||||
.whenComplete((r, e) -> {
|
.whenComplete((r, e) -> {
|
||||||
if (e != null) {
|
if (e != null) {
|
||||||
|
@ -18,6 +18,11 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase;
|
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.IOException;
|
||||||
import java.io.PrintStream;
|
import java.io.PrintStream;
|
||||||
import java.lang.reflect.Constructor;
|
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.Get;
|
||||||
import org.apache.hadoop.hbase.client.Increment;
|
import org.apache.hadoop.hbase.client.Increment;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
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.Result;
|
||||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
import org.apache.hadoop.hbase.client.RowMutations;
|
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.mapreduce.TableMapReduceUtil;
|
||||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||||
import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
|
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.HBaseHTraceConfiguration;
|
||||||
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||||
import org.apache.hadoop.hbase.trace.TraceUtil;
|
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.htrace.core.TraceScope;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
import com.codahale.metrics.Histogram;
|
import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects;
|
||||||
import com.codahale.metrics.UniformReservoir;
|
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import com.fasterxml.jackson.databind.MapperFeature;
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Script used evaluating HBase performance and scalability. Runs a HBase
|
* 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 {
|
static abstract class AsyncTableTest extends AsyncTest {
|
||||||
protected RawAsyncTable table;
|
protected AsyncTable<?> table;
|
||||||
|
|
||||||
AsyncTableTest(AsyncConnection con, TestOptions options, Status status) {
|
AsyncTableTest(AsyncConnection con, TestOptions options, Status status) {
|
||||||
super(con, options, status);
|
super(con, options, status);
|
||||||
@ -1310,7 +1310,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
void onStartup() throws IOException {
|
void onStartup() throws IOException {
|
||||||
this.table = connection.getRawTable(TableName.valueOf(opts.tableName));
|
this.table = connection.getTable(TableName.valueOf(opts.tableName));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -1435,7 +1435,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
|
|||||||
|
|
||||||
static class AsyncScanTest extends AsyncTableTest {
|
static class AsyncScanTest extends AsyncTableTest {
|
||||||
private ResultScanner testScanner;
|
private ResultScanner testScanner;
|
||||||
private AsyncTable asyncTable;
|
private AsyncTable<?> asyncTable;
|
||||||
|
|
||||||
AsyncScanTest(AsyncConnection con, TestOptions options, Status status) {
|
AsyncScanTest(AsyncConnection con, TestOptions options, Status status) {
|
||||||
super(con, options, status);
|
super(con, options, status);
|
||||||
|
@ -24,6 +24,7 @@ import java.io.IOException;
|
|||||||
import java.io.UncheckedIOException;
|
import java.io.UncheckedIOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.concurrent.ForkJoinPool;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
@ -63,7 +64,7 @@ public abstract class AbstractTestAsyncTableScan {
|
|||||||
TEST_UTIL.createTable(TABLE_NAME, FAMILY, splitKeys);
|
TEST_UTIL.createTable(TABLE_NAME, FAMILY, splitKeys);
|
||||||
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
||||||
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
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)))
|
.mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
|
||||||
.addColumn(FAMILY, CQ1, Bytes.toBytes(i)).addColumn(FAMILY, CQ2, Bytes.toBytes(i * i)))
|
.addColumn(FAMILY, CQ1, Bytes.toBytes(i)).addColumn(FAMILY, CQ2, Bytes.toBytes(i * i)))
|
||||||
.collect(Collectors.toList())).get();
|
.collect(Collectors.toList())).get();
|
||||||
@ -92,7 +93,15 @@ public abstract class AbstractTestAsyncTableScan {
|
|||||||
return new Scan().setBatch(1).setMaxResultSize(1);
|
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),
|
return Arrays.asList(Pair.newPair("normal", AbstractTestAsyncTableScan::createNormalScan),
|
||||||
Pair.newPair("batch", AbstractTestAsyncTableScan::createBatchScan),
|
Pair.newPair("batch", AbstractTestAsyncTableScan::createBatchScan),
|
||||||
Pair.newPair("smallResultSize", AbstractTestAsyncTableScan::createSmallResultSizeScan),
|
Pair.newPair("smallResultSize", AbstractTestAsyncTableScan::createSmallResultSizeScan),
|
||||||
@ -100,6 +109,25 @@ public abstract class AbstractTestAsyncTableScan {
|
|||||||
AbstractTestAsyncTableScan::createBatchSmallResultSizeScan));
|
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 Scan createScan();
|
||||||
|
|
||||||
protected abstract List<Result> doScan(Scan scan) throws Exception;
|
protected abstract List<Result> doScan(Scan scan) throws Exception;
|
||||||
@ -121,10 +149,11 @@ public abstract class AbstractTestAsyncTableScan {
|
|||||||
List<Result> results = doScan(createScan());
|
List<Result> results = doScan(createScan());
|
||||||
// make sure all scanners are closed at RS side
|
// make sure all scanners are closed at RS side
|
||||||
TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
|
TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
|
||||||
.forEach(rs -> assertEquals(
|
.forEach(
|
||||||
"The scanner count of " + rs.getServerName() + " is " +
|
rs -> assertEquals(
|
||||||
|
"The scanner count of " + rs.getServerName() + " is " +
|
||||||
rs.getRSRpcServices().getScannersCount(),
|
rs.getRSRpcServices().getScannersCount(),
|
||||||
0, rs.getRSRpcServices().getScannersCount()));
|
0, rs.getRSRpcServices().getScannersCount()));
|
||||||
assertEquals(COUNT, results.size());
|
assertEquals(COUNT, results.size());
|
||||||
IntStream.range(0, COUNT).forEach(i -> {
|
IntStream.range(0, COUNT).forEach(i -> {
|
||||||
Result result = results.get(i);
|
Result result = results.get(i);
|
||||||
@ -150,7 +179,7 @@ public abstract class AbstractTestAsyncTableScan {
|
|||||||
public void testScanNoStopKey() throws Exception {
|
public void testScanNoStopKey() throws Exception {
|
||||||
int start = 345;
|
int start = 345;
|
||||||
List<Result> results =
|
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());
|
assertEquals(COUNT - start, results.size());
|
||||||
IntStream.range(0, COUNT - start).forEach(i -> assertResultEquals(results.get(i), start + i));
|
IntStream.range(0, COUNT - start).forEach(i -> assertResultEquals(results.get(i), start + i));
|
||||||
}
|
}
|
||||||
@ -169,16 +198,16 @@ public abstract class AbstractTestAsyncTableScan {
|
|||||||
try {
|
try {
|
||||||
doScan(createScan().addFamily(Bytes.toBytes("WrongColumnFamily")));
|
doScan(createScan().addFamily(Bytes.toBytes("WrongColumnFamily")));
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
assertTrue(e instanceof NoSuchColumnFamilyException
|
assertTrue(e instanceof NoSuchColumnFamilyException ||
|
||||||
|| e.getCause() instanceof NoSuchColumnFamilyException);
|
e.getCause() instanceof NoSuchColumnFamilyException);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void testScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
|
private void testScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
|
||||||
int limit) throws Exception {
|
int limit) throws Exception {
|
||||||
Scan scan =
|
Scan scan =
|
||||||
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
|
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
|
||||||
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive);
|
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive);
|
||||||
if (limit > 0) {
|
if (limit > 0) {
|
||||||
scan.setLimit(limit);
|
scan.setLimit(limit);
|
||||||
}
|
}
|
||||||
@ -195,9 +224,9 @@ public abstract class AbstractTestAsyncTableScan {
|
|||||||
|
|
||||||
private void testReversedScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
|
private void testReversedScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
|
||||||
int limit) throws Exception {
|
int limit) throws Exception {
|
||||||
Scan scan = createScan()
|
Scan scan =
|
||||||
.withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
|
createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
|
||||||
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive).setReversed(true);
|
.withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive).setReversed(true);
|
||||||
if (limit > 0) {
|
if (limit > 0) {
|
||||||
scan.setLimit(limit);
|
scan.setLimit(limit);
|
||||||
}
|
}
|
||||||
|
@ -25,7 +25,12 @@ import java.util.Queue;
|
|||||||
|
|
||||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
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;
|
private ScanMetrics scanMetrics;
|
||||||
|
|
@ -96,7 +96,7 @@ public class TestAsyncBufferMutator {
|
|||||||
}
|
}
|
||||||
// mutator.close will call mutator.flush automatically so all tasks should have been done.
|
// mutator.close will call mutator.flush automatically so all tasks should have been done.
|
||||||
futures.forEach(f -> f.join());
|
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())
|
IntStream.range(0, COUNT).mapToObj(i -> new Get(Bytes.toBytes(i))).map(g -> table.get(g).join())
|
||||||
.forEach(r -> {
|
.forEach(r -> {
|
||||||
assertArrayEquals(VALUE, r.getValue(CF, CQ));
|
assertArrayEquals(VALUE, r.getValue(CF, CQ));
|
||||||
|
@ -212,7 +212,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
|
|||||||
|
|
||||||
private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws Exception {
|
private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws Exception {
|
||||||
createTableWithDefaultConf(tableName);
|
createTableWithDefaultConf(tableName);
|
||||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
|
||||||
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||||
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
|
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)));
|
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);
|
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(table);
|
||||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
||||||
admin.createTable(builder.build(), Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), 16).join();
|
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<>();
|
List<Put> puts = new ArrayList<>();
|
||||||
for (byte[] row : HBaseTestingUtility.ROWS) {
|
for (byte[] row : HBaseTestingUtility.ROWS) {
|
||||||
puts.add(new Put(row).addColumn(FAMILY, Bytes.toBytes("q"), Bytes.toBytes("v")));
|
puts.add(new Put(row).addColumn(FAMILY, Bytes.toBytes("q"), Bytes.toBytes("v")));
|
||||||
|
@ -69,12 +69,12 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
|
TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
|
||||||
AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(tableName);
|
AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(tableName);
|
||||||
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm")).get();
|
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm")).get();
|
||||||
RegionInfo region = regionLocation.getRegionInfo();
|
RegionInfo region = regionLocation.getRegion();
|
||||||
byte[] regionName = regionLocation.getRegionInfo().getRegionName();
|
byte[] regionName = regionLocation.getRegion().getRegionName();
|
||||||
HRegionLocation location = rawAdmin.getRegionLocation(regionName).get();
|
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();
|
location = rawAdmin.getRegionLocation(region.getEncodedNameAsBytes()).get();
|
||||||
assertTrue(Bytes.equals(regionName, location.getRegionInfo().getRegionName()));
|
assertTrue(Bytes.equals(regionName, location.getRegion().getRegionName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -252,7 +252,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
.filter(rs -> rs.getServerName().equals(serverName)).findFirst().get();
|
.filter(rs -> rs.getServerName().equals(serverName)).findFirst().get();
|
||||||
|
|
||||||
// write a put into the specific region
|
// 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")))
|
.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-1")))
|
||||||
.join();
|
.join();
|
||||||
Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0);
|
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);
|
Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize(), 0);
|
||||||
|
|
||||||
// write another put into the specific region
|
// 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")))
|
.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-2")))
|
||||||
.join();
|
.join();
|
||||||
Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0);
|
Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0);
|
||||||
@ -288,7 +288,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
final int rows = 10000;
|
final int rows = 10000;
|
||||||
loadData(tableName, families, rows);
|
loadData(tableName, families, rows);
|
||||||
|
|
||||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||||
List<HRegionLocation> regionLocations =
|
List<HRegionLocation> regionLocations =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
int originalCount = regionLocations.size();
|
int originalCount = regionLocations.size();
|
||||||
@ -319,7 +319,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
byte[][] families = { FAMILY };
|
byte[][] families = { FAMILY };
|
||||||
loadData(tableName, families, 1000);
|
loadData(tableName, families, 1000);
|
||||||
|
|
||||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||||
List<HRegionLocation> regionLocations =
|
List<HRegionLocation> regionLocations =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
int originalCount = regionLocations.size();
|
int originalCount = regionLocations.size();
|
||||||
@ -364,7 +364,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
|
byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
|
||||||
createTableWithDefaultConf(tableName, splitRows);
|
createTableWithDefaultConf(tableName, splitRows);
|
||||||
|
|
||||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||||
List<HRegionLocation> regionLocations =
|
List<HRegionLocation> regionLocations =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
RegionInfo regionA;
|
RegionInfo regionA;
|
||||||
@ -372,16 +372,16 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
|
|
||||||
// merge with full name
|
// merge with full name
|
||||||
assertEquals(3, regionLocations.size());
|
assertEquals(3, regionLocations.size());
|
||||||
regionA = regionLocations.get(0).getRegionInfo();
|
regionA = regionLocations.get(0).getRegion();
|
||||||
regionB = regionLocations.get(1).getRegionInfo();
|
regionB = regionLocations.get(1).getRegion();
|
||||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||||
|
|
||||||
regionLocations =
|
regionLocations =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
assertEquals(2, regionLocations.size());
|
assertEquals(2, regionLocations.size());
|
||||||
// merge with encoded name
|
// merge with encoded name
|
||||||
regionA = regionLocations.get(0).getRegionInfo();
|
regionA = regionLocations.get(0).getRegion();
|
||||||
regionB = regionLocations.get(1).getRegionInfo();
|
regionB = regionLocations.get(1).getRegion();
|
||||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||||
|
|
||||||
regionLocations =
|
regionLocations =
|
||||||
@ -404,12 +404,12 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
// create table
|
// create table
|
||||||
createTableWithDefaultConf(tableName);
|
createTableWithDefaultConf(tableName);
|
||||||
|
|
||||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||||
List<HRegionLocation> regionLocations =
|
List<HRegionLocation> regionLocations =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
assertEquals(1, regionLocations.size());
|
assertEquals(1, regionLocations.size());
|
||||||
|
|
||||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
|
||||||
List<Put> puts = new ArrayList<>();
|
List<Put> puts = new ArrayList<>();
|
||||||
for (int i = 0; i < rowCount; i++) {
|
for (int i = 0; i < rowCount; i++) {
|
||||||
Put put = new Put(Bytes.toBytes(i));
|
Put put = new Put(Bytes.toBytes(i));
|
||||||
@ -420,9 +420,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||||||
|
|
||||||
if (isSplitRegion) {
|
if (isSplitRegion) {
|
||||||
if (splitPoint == null) {
|
if (splitPoint == null) {
|
||||||
admin.splitRegion(regionLocations.get(0).getRegionInfo().getRegionName()).get();
|
admin.splitRegion(regionLocations.get(0).getRegion().getRegionName()).get();
|
||||||
} else {
|
} else {
|
||||||
admin.splitRegion(regionLocations.get(0).getRegionInfo().getRegionName(), splitPoint).get();
|
admin.splitRegion(regionLocations.get(0).getRegion().getRegionName(), splitPoint).get();
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (splitPoint == null) {
|
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,
|
private static void loadData(final TableName tableName, final byte[][] families, final int rows,
|
||||||
final int flushes) throws IOException {
|
final int flushes) throws IOException {
|
||||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
|
||||||
List<Put> puts = new ArrayList<>(rows);
|
List<Put> puts = new ArrayList<>(rows);
|
||||||
byte[] qualifier = Bytes.toBytes("val");
|
byte[] qualifier = Bytes.toBytes("val");
|
||||||
for (int i = 0; i < flushes; i++) {
|
for (int i = 0; i < flushes; i++) {
|
||||||
|
@ -63,7 +63,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
|
|||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUpBeforeClass() throws Exception {
|
public static void setUpBeforeClass() throws Exception {
|
||||||
TEST_UTIL.startMiniCluster(2);
|
TEST_UTIL.startMiniCluster(2);
|
||||||
TEST_UTIL.getAdmin().setBalancerRunning(false, true);
|
TEST_UTIL.getAdmin().balancerSwitch(false, true);
|
||||||
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
|
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
|
||||||
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
||||||
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
|
AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
|
||||||
@ -81,15 +81,15 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
|
|||||||
public void testRegionMove() throws InterruptedException, ExecutionException, IOException {
|
public void testRegionMove() throws InterruptedException, ExecutionException, IOException {
|
||||||
// This will leave a cached entry in location cache
|
// This will leave a cached entry in location cache
|
||||||
HRegionLocation loc = CONN.getRegionLocator(TABLE_NAME).getRegionLocation(ROW).get();
|
HRegionLocation loc = CONN.getRegionLocator(TABLE_NAME).getRegionLocation(ROW).get();
|
||||||
int index = TEST_UTIL.getHBaseCluster().getServerWith(loc.getRegionInfo().getRegionName());
|
int index = TEST_UTIL.getHBaseCluster().getServerWith(loc.getRegion().getRegionName());
|
||||||
TEST_UTIL.getAdmin().move(loc.getRegionInfo().getEncodedNameAsBytes(), Bytes.toBytes(
|
TEST_UTIL.getAdmin().move(loc.getRegion().getEncodedNameAsBytes(), Bytes.toBytes(
|
||||||
TEST_UTIL.getHBaseCluster().getRegionServer(1 - index).getServerName().getServerName()));
|
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();
|
.setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(30).build();
|
||||||
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
||||||
|
|
||||||
// move back
|
// move back
|
||||||
TEST_UTIL.getAdmin().move(loc.getRegionInfo().getEncodedNameAsBytes(),
|
TEST_UTIL.getAdmin().move(loc.getRegion().getEncodedNameAsBytes(),
|
||||||
Bytes.toBytes(loc.getServerName().getServerName()));
|
Bytes.toBytes(loc.getServerName().getServerName()));
|
||||||
Result result = table.get(new Get(ROW).addColumn(FAMILY, QUALIFIER)).get();
|
Result result = table.get(new Get(ROW).addColumn(FAMILY, QUALIFIER)).get();
|
||||||
assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER));
|
assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER));
|
||||||
@ -166,7 +166,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
|
|||||||
return mockedLocator;
|
return mockedLocator;
|
||||||
}
|
}
|
||||||
}) {
|
}) {
|
||||||
RawAsyncTable table = mockedConn.getRawTableBuilder(TABLE_NAME)
|
AsyncTable<?> table = mockedConn.getTableBuilder(TABLE_NAME)
|
||||||
.setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(5).build();
|
.setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(5).build();
|
||||||
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
||||||
assertTrue(errorTriggered.get());
|
assertTrue(errorTriggered.get());
|
||||||
|
@ -78,13 +78,13 @@ public class TestAsyncTable {
|
|||||||
private byte[] row;
|
private byte[] row;
|
||||||
|
|
||||||
@Parameter
|
@Parameter
|
||||||
public Supplier<AsyncTableBase> getTable;
|
public Supplier<AsyncTable<?>> getTable;
|
||||||
|
|
||||||
private static RawAsyncTable getRawTable() {
|
private static AsyncTable<?> getRawTable() {
|
||||||
return ASYNC_CONN.getRawTable(TABLE_NAME);
|
return ASYNC_CONN.getTable(TABLE_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static AsyncTable getTable() {
|
private static AsyncTable<?> getTable() {
|
||||||
return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
|
return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -115,7 +115,7 @@ public class TestAsyncTable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSimple() throws Exception {
|
public void testSimple() throws Exception {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
||||||
assertTrue(table.exists(new Get(row).addColumn(FAMILY, QUALIFIER)).get());
|
assertTrue(table.exists(new Get(row).addColumn(FAMILY, QUALIFIER)).get());
|
||||||
Result result = table.get(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
|
@Test
|
||||||
public void testSimpleMultiple() throws Exception {
|
public void testSimpleMultiple() throws Exception {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
int count = 100;
|
int count = 100;
|
||||||
CountDownLatch putLatch = new CountDownLatch(count);
|
CountDownLatch putLatch = new CountDownLatch(count);
|
||||||
IntStream.range(0, count).forEach(
|
IntStream.range(0, count).forEach(
|
||||||
@ -176,7 +176,7 @@ public class TestAsyncTable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIncrement() throws InterruptedException, ExecutionException {
|
public void testIncrement() throws InterruptedException, ExecutionException {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
int count = 100;
|
int count = 100;
|
||||||
CountDownLatch latch = new CountDownLatch(count);
|
CountDownLatch latch = new CountDownLatch(count);
|
||||||
AtomicLong sum = new AtomicLong(0L);
|
AtomicLong sum = new AtomicLong(0L);
|
||||||
@ -193,7 +193,7 @@ public class TestAsyncTable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAppend() throws InterruptedException, ExecutionException {
|
public void testAppend() throws InterruptedException, ExecutionException {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
int count = 10;
|
int count = 10;
|
||||||
CountDownLatch latch = new CountDownLatch(count);
|
CountDownLatch latch = new CountDownLatch(count);
|
||||||
char suffix = ':';
|
char suffix = ':';
|
||||||
@ -216,7 +216,7 @@ public class TestAsyncTable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCheckAndPut() throws InterruptedException, ExecutionException {
|
public void testCheckAndPut() throws InterruptedException, ExecutionException {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
AtomicInteger successCount = new AtomicInteger(0);
|
AtomicInteger successCount = new AtomicInteger(0);
|
||||||
AtomicInteger successIndex = new AtomicInteger(-1);
|
AtomicInteger successIndex = new AtomicInteger(-1);
|
||||||
int count = 10;
|
int count = 10;
|
||||||
@ -238,7 +238,7 @@ public class TestAsyncTable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCheckAndDelete() throws InterruptedException, ExecutionException {
|
public void testCheckAndDelete() throws InterruptedException, ExecutionException {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
int count = 10;
|
int count = 10;
|
||||||
CountDownLatch putLatch = new CountDownLatch(count + 1);
|
CountDownLatch putLatch = new CountDownLatch(count + 1);
|
||||||
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).thenRun(() -> putLatch.countDown());
|
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).thenRun(() -> putLatch.countDown());
|
||||||
@ -275,7 +275,7 @@ public class TestAsyncTable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMutateRow() throws InterruptedException, ExecutionException, IOException {
|
public void testMutateRow() throws InterruptedException, ExecutionException, IOException {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
RowMutations mutation = new RowMutations(row);
|
RowMutations mutation = new RowMutations(row);
|
||||||
mutation.add(new Put(row).addColumn(FAMILY, concat(QUALIFIER, 1), VALUE));
|
mutation.add(new Put(row).addColumn(FAMILY, concat(QUALIFIER, 1), VALUE));
|
||||||
table.mutateRow(mutation).get();
|
table.mutateRow(mutation).get();
|
||||||
@ -293,7 +293,7 @@ public class TestAsyncTable {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCheckAndMutate() throws InterruptedException, ExecutionException {
|
public void testCheckAndMutate() throws InterruptedException, ExecutionException {
|
||||||
AsyncTableBase table = getTable.get();
|
AsyncTable<?> table = getTable.get();
|
||||||
int count = 10;
|
int count = 10;
|
||||||
CountDownLatch putLatch = new CountDownLatch(count + 1);
|
CountDownLatch putLatch = new CountDownLatch(count + 1);
|
||||||
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).thenRun(() -> putLatch.countDown());
|
table.put(new Put(row).addColumn(FAMILY, QUALIFIER, VALUE)).thenRun(() -> putLatch.countDown());
|
||||||
|
@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@ -39,7 +38,6 @@ import java.util.regex.Pattern;
|
|||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
|
||||||
import org.apache.hadoop.hbase.HRegionLocation;
|
import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
@ -61,7 +59,7 @@ import org.junit.runners.Parameterized;
|
|||||||
* Class to test asynchronous table admin operations.
|
* Class to test asynchronous table admin operations.
|
||||||
*/
|
*/
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
@Category({LargeTests.class, ClientTests.class})
|
@Category({ LargeTests.class, ClientTests.class })
|
||||||
public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -153,25 +151,25 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private TableState.State getStateFromMeta(TableName table) throws Exception {
|
private TableState.State getStateFromMeta(TableName table) throws Exception {
|
||||||
Optional<TableState> state = AsyncMetaTableAccessor.getTableState(
|
Optional<TableState> state = AsyncMetaTableAccessor
|
||||||
ASYNC_CONN.getRawTable(TableName.META_TABLE_NAME), table).get();
|
.getTableState(ASYNC_CONN.getTable(TableName.META_TABLE_NAME), table).get();
|
||||||
assertTrue(state.isPresent());
|
assertTrue(state.isPresent());
|
||||||
return state.get().getState();
|
return state.get().getState();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateTableNumberOfRegions() throws Exception {
|
public void testCreateTableNumberOfRegions() throws Exception {
|
||||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||||
|
|
||||||
createTableWithDefaultConf(tableName);
|
createTableWithDefaultConf(tableName);
|
||||||
List<HRegionLocation> regionLocations =
|
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());
|
assertEquals("Table should have only 1 region", 1, regionLocations.size());
|
||||||
|
|
||||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "_2");
|
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "_2");
|
||||||
createTableWithDefaultConf(tableName2, new byte[][] { new byte[] { 42 } });
|
createTableWithDefaultConf(tableName2, new byte[][] { new byte[] { 42 } });
|
||||||
regionLocations =
|
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());
|
assertEquals("Table should have only 2 region", 2, regionLocations.size());
|
||||||
|
|
||||||
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "_3");
|
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "_3");
|
||||||
@ -179,7 +177,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
||||||
admin.createTable(builder.build(), "a".getBytes(), "z".getBytes(), 3).join();
|
admin.createTable(builder.build(), "a".getBytes(), "z".getBytes(), 3).join();
|
||||||
regionLocations =
|
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());
|
assertEquals("Table should have only 3 region", 3, regionLocations.size());
|
||||||
|
|
||||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
|
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
|
||||||
@ -197,15 +195,15 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
|
||||||
admin.createTable(builder.build(), new byte[] { 1 }, new byte[] { 127 }, 16).join();
|
admin.createTable(builder.build(), new byte[] { 1 }, new byte[] { 127 }, 16).join();
|
||||||
regionLocations =
|
regionLocations =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName5)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName5)).get();
|
||||||
assertEquals("Table should have 16 region", 16, regionLocations.size());
|
assertEquals("Table should have 16 region", 16, regionLocations.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateTableWithRegions() throws Exception {
|
public void testCreateTableWithRegions() throws Exception {
|
||||||
byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
|
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[] { 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[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 }, };
|
||||||
int expectedRegions = splitKeys.length + 1;
|
int expectedRegions = splitKeys.length + 1;
|
||||||
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
|
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
|
||||||
createTableWithDefaultConf(tableName, splitKeys);
|
createTableWithDefaultConf(tableName, splitKeys);
|
||||||
@ -213,9 +211,9 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
|
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
|
||||||
assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
|
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 =
|
List<HRegionLocation> regions =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
Iterator<HRegionLocation> hris = regions.iterator();
|
Iterator<HRegionLocation> hris = regions.iterator();
|
||||||
|
|
||||||
assertEquals(
|
assertEquals(
|
||||||
@ -223,36 +221,36 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
expectedRegions, regions.size());
|
expectedRegions, regions.size());
|
||||||
System.err.println("Found " + regions.size() + " regions");
|
System.err.println("Found " + regions.size() + " regions");
|
||||||
|
|
||||||
HRegionInfo hri;
|
RegionInfo hri;
|
||||||
hris = regions.iterator();
|
hris = regions.iterator();
|
||||||
hri = hris.next().getRegionInfo();
|
hri = hris.next().getRegion();
|
||||||
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
|
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[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.getStartKey(), splitKeys[0]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1]));
|
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.getStartKey(), splitKeys[1]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2]));
|
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.getStartKey(), splitKeys[2]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3]));
|
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.getStartKey(), splitKeys[3]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4]));
|
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.getStartKey(), splitKeys[4]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5]));
|
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.getStartKey(), splitKeys[5]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6]));
|
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.getStartKey(), splitKeys[6]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7]));
|
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.getStartKey(), splitKeys[7]));
|
||||||
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
|
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
|
||||||
hri = hris.next().getRegionInfo();
|
hri = hris.next().getRegion();
|
||||||
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
|
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
|
||||||
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
|
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
|
||||||
if (tablesOnMaster) {
|
if (tablesOnMaster) {
|
||||||
@ -274,41 +272,41 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
||||||
|
|
||||||
regions =
|
regions =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||||
expectedRegions, regions.size());
|
expectedRegions, regions.size());
|
||||||
System.err.println("Found " + regions.size() + " regions");
|
System.err.println("Found " + regions.size() + " regions");
|
||||||
|
|
||||||
hris = regions.iterator();
|
hris = regions.iterator();
|
||||||
hri = hris.next().getRegionInfo();
|
hri = hris.next().getRegion();
|
||||||
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
|
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 }));
|
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.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 }));
|
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.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 }));
|
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.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 }));
|
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.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 }));
|
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.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 }));
|
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.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 }));
|
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.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 }));
|
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.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 }));
|
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(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
|
||||||
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
|
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
|
||||||
if (tablesOnMaster) {
|
if (tablesOnMaster) {
|
||||||
@ -327,7 +325,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
||||||
|
|
||||||
regions =
|
regions =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||||
expectedRegions, regions.size());
|
expectedRegions, regions.size());
|
||||||
@ -339,8 +337,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
|
|
||||||
// Try an invalid case where there are duplicate split keys
|
// Try an invalid case where there are duplicate split keys
|
||||||
splitKeys = new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 },
|
splitKeys = new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 },
|
||||||
new byte[] { 3, 3, 3 }, new byte[] { 2, 2, 2 } };
|
new byte[] { 3, 3, 3 }, new byte[] { 2, 2, 2 } };
|
||||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");;
|
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
|
||||||
try {
|
try {
|
||||||
createTableWithDefaultConf(tableName4, splitKeys);
|
createTableWithDefaultConf(tableName4, splitKeys);
|
||||||
fail("Should not be able to create this table because of " + "duplicate split keys");
|
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 {
|
throws IOException {
|
||||||
int numRS = ((ClusterConnection) TEST_UTIL.getConnection()).getCurrentNrHRS();
|
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) -> {
|
regions.stream().forEach((loc) -> {
|
||||||
ServerName server = loc.getServerName();
|
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) {
|
if (numRS >= 2) {
|
||||||
// Ignore the master region server,
|
// Ignore the master region server,
|
||||||
@ -424,7 +422,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
|
|
||||||
// Create & Fill the table
|
// Create & Fill the table
|
||||||
createTableWithDefaultConf(tableName, splitKeys);
|
createTableWithDefaultConf(tableName, splitKeys);
|
||||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
|
||||||
int expectedRows = 10;
|
int expectedRows = 10;
|
||||||
for (int i = 0; i < expectedRows; i++) {
|
for (int i = 0; i < expectedRows; i++) {
|
||||||
byte[] data = Bytes.toBytes(String.valueOf(i));
|
byte[] data = Bytes.toBytes(String.valueOf(i));
|
||||||
@ -449,7 +447,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
@Test
|
@Test
|
||||||
public void testDisableAndEnableTable() throws Exception {
|
public void testDisableAndEnableTable() throws Exception {
|
||||||
createTableWithDefaultConf(tableName);
|
createTableWithDefaultConf(tableName);
|
||||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
|
||||||
final byte[] row = Bytes.toBytes("row");
|
final byte[] row = Bytes.toBytes("row");
|
||||||
final byte[] qualifier = Bytes.toBytes("qualifier");
|
final byte[] qualifier = Bytes.toBytes("qualifier");
|
||||||
final byte[] value = Bytes.toBytes("value");
|
final byte[] value = Bytes.toBytes("value");
|
||||||
@ -502,8 +500,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "2");
|
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "2");
|
||||||
createTableWithDefaultConf(tableName1);
|
createTableWithDefaultConf(tableName1);
|
||||||
createTableWithDefaultConf(tableName2);
|
createTableWithDefaultConf(tableName2);
|
||||||
RawAsyncTable table1 = ASYNC_CONN.getRawTable(tableName1);
|
AsyncTable<?> table1 = ASYNC_CONN.getTable(tableName1);
|
||||||
RawAsyncTable table2 = ASYNC_CONN.getRawTable(tableName1);
|
AsyncTable<?> table2 = ASYNC_CONN.getTable(tableName1);
|
||||||
|
|
||||||
final byte[] row = Bytes.toBytes("row");
|
final byte[] row = Bytes.toBytes("row");
|
||||||
final byte[] qualifier = Bytes.toBytes("qualifier");
|
final byte[] qualifier = Bytes.toBytes("qualifier");
|
||||||
@ -517,8 +515,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
table1.get(get).get();
|
table1.get(get).get();
|
||||||
table2.get(get).get();
|
table2.get(get).get();
|
||||||
|
|
||||||
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false)
|
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false).get()
|
||||||
.get().forEach(t -> admin.disableTable(t).join());
|
.forEach(t -> admin.disableTable(t).join());
|
||||||
|
|
||||||
// Test that tables are disabled
|
// Test that tables are disabled
|
||||||
get = new Get(row);
|
get = new Get(row);
|
||||||
@ -541,8 +539,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName1));
|
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName1));
|
||||||
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName2));
|
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName2));
|
||||||
|
|
||||||
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false)
|
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false).get()
|
||||||
.get().forEach(t -> admin.enableTable(t).join());
|
.forEach(t -> admin.enableTable(t).join());
|
||||||
|
|
||||||
// Test that tables are enabled
|
// Test that tables are enabled
|
||||||
try {
|
try {
|
||||||
@ -562,16 +560,15 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testEnableTableRetainAssignment() throws Exception {
|
public void testEnableTableRetainAssignment() throws Exception {
|
||||||
byte[][] splitKeys =
|
byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
|
||||||
{ 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[] { 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[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 } };
|
|
||||||
int expectedRegions = splitKeys.length + 1;
|
int expectedRegions = splitKeys.length + 1;
|
||||||
createTableWithDefaultConf(tableName, splitKeys);
|
createTableWithDefaultConf(tableName, splitKeys);
|
||||||
|
|
||||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
|
||||||
List<HRegionLocation> regions =
|
List<HRegionLocation> regions =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||||
expectedRegions, regions.size());
|
expectedRegions, regions.size());
|
||||||
@ -582,7 +579,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
admin.enableTable(tableName).join();
|
admin.enableTable(tableName).join();
|
||||||
|
|
||||||
List<HRegionLocation> regions2 =
|
List<HRegionLocation> regions2 =
|
||||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||||
// Check the assignment.
|
// Check the assignment.
|
||||||
assertEquals(regions.size(), regions2.size());
|
assertEquals(regions.size(), regions2.size());
|
||||||
assertTrue(regions2.containsAll(regions));
|
assertTrue(regions2.containsAll(regions));
|
||||||
@ -611,8 +608,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
verifyTableDescriptor(tableName, FAMILY_0);
|
verifyTableDescriptor(tableName, FAMILY_0);
|
||||||
|
|
||||||
// Modify the table removing one family and verify the descriptor
|
// Modify the table removing one family and verify the descriptor
|
||||||
admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.of(FAMILY_1))
|
admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.of(FAMILY_1)).join();
|
||||||
.join();
|
|
||||||
verifyTableDescriptor(tableName, FAMILY_0, FAMILY_1);
|
verifyTableDescriptor(tableName, FAMILY_0, FAMILY_1);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -632,8 +628,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
|
|
||||||
try {
|
try {
|
||||||
// Add same column family again - expect failure
|
// Add same column family again - expect failure
|
||||||
this.admin.addColumnFamily(tableName,
|
this.admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.of(FAMILY_1)).join();
|
||||||
ColumnFamilyDescriptorBuilder.of(FAMILY_1)).join();
|
|
||||||
Assert.fail("Delete a non-exist column family should fail");
|
Assert.fail("Delete a non-exist column family should fail");
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// Expected.
|
// Expected.
|
||||||
@ -731,8 +726,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
// Verify descriptor from HDFS
|
// Verify descriptor from HDFS
|
||||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||||
TableDescriptor td =
|
TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||||
FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
|
||||||
verifyTableDescriptor(td, tableName, families);
|
verifyTableDescriptor(td, tableName, families);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -768,7 +762,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
@Test
|
@Test
|
||||||
public void testCompactionTimestamps() throws Exception {
|
public void testCompactionTimestamps() throws Exception {
|
||||||
createTableWithDefaultConf(tableName);
|
createTableWithDefaultConf(tableName);
|
||||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
AsyncTable<?> table = ASYNC_CONN.getTable(tableName);
|
||||||
Optional<Long> ts = admin.getLastMajorCompactionTimestamp(tableName).get();
|
Optional<Long> ts = admin.getLastMajorCompactionTimestamp(tableName).get();
|
||||||
assertFalse(ts.isPresent());
|
assertFalse(ts.isPresent());
|
||||||
Put p = new Put(Bytes.toBytes("row1"));
|
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
|
// still 0, we flushed a file, but no major compaction happened
|
||||||
assertFalse(ts.isPresent());
|
assertFalse(ts.isPresent());
|
||||||
|
|
||||||
byte[] regionName =
|
byte[] regionName = ASYNC_CONN.getRegionLocator(tableName)
|
||||||
ASYNC_CONN.getRegionLocator(tableName).getRegionLocation(Bytes.toBytes("row1")).get()
|
.getRegionLocation(Bytes.toBytes("row1")).get().getRegion().getRegionName();
|
||||||
.getRegionInfo().getRegionName();
|
|
||||||
Optional<Long> ts1 = admin.getLastMajorCompactionTimestampForRegion(regionName).get();
|
Optional<Long> ts1 = admin.getLastMajorCompactionTimestampForRegion(regionName).get();
|
||||||
assertFalse(ts1.isPresent());
|
assertFalse(ts1.isPresent());
|
||||||
p = new Put(Bytes.toBytes("row2"));
|
p = new Put(Bytes.toBytes("row2"));
|
||||||
@ -823,7 +816,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Sleep to wait region server report
|
// 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();
|
ts = admin.getLastMajorCompactionTimestamp(tableName).get();
|
||||||
// after a compaction our earliest timestamp will have progressed forward
|
// after a compaction our earliest timestamp will have progressed forward
|
||||||
|
@ -32,13 +32,15 @@ import java.util.Optional;
|
|||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ForkJoinPool;
|
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.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||||
@ -80,20 +82,20 @@ public class TestAsyncTableBatch {
|
|||||||
public String tableType;
|
public String tableType;
|
||||||
|
|
||||||
@Parameter(1)
|
@Parameter(1)
|
||||||
public Function<TableName, AsyncTableBase> tableGetter;
|
public Function<TableName, AsyncTable<?>> tableGetter;
|
||||||
|
|
||||||
private static RawAsyncTable getRawTable(TableName tableName) {
|
private static AsyncTable<?> getRawTable(TableName tableName) {
|
||||||
return CONN.getRawTable(tableName);
|
return CONN.getTable(tableName);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static AsyncTable getTable(TableName tableName) {
|
private static AsyncTable<?> getTable(TableName tableName) {
|
||||||
return CONN.getTable(tableName, ForkJoinPool.commonPool());
|
return CONN.getTable(tableName, ForkJoinPool.commonPool());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Parameters(name = "{index}: type={0}")
|
@Parameters(name = "{index}: type={0}")
|
||||||
public static List<Object[]> params() {
|
public static List<Object[]> params() {
|
||||||
Function<TableName, AsyncTableBase> rawTableGetter = TestAsyncTableBatch::getRawTable;
|
Function<TableName, AsyncTable<?>> rawTableGetter = TestAsyncTableBatch::getRawTable;
|
||||||
Function<TableName, AsyncTableBase> tableGetter = TestAsyncTableBatch::getTable;
|
Function<TableName, AsyncTable<?>> tableGetter = TestAsyncTableBatch::getTable;
|
||||||
return Arrays.asList(new Object[] { "raw", rawTableGetter },
|
return Arrays.asList(new Object[] { "raw", rawTableGetter },
|
||||||
new Object[] { "normal", tableGetter });
|
new Object[] { "normal", tableGetter });
|
||||||
}
|
}
|
||||||
@ -134,18 +136,15 @@ public class TestAsyncTableBatch {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void test() throws InterruptedException, ExecutionException, IOException {
|
public void test()
|
||||||
AsyncTableBase table = tableGetter.apply(TABLE_NAME);
|
throws InterruptedException, ExecutionException, IOException, TimeoutException {
|
||||||
|
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
|
||||||
table.putAll(IntStream.range(0, COUNT)
|
table.putAll(IntStream.range(0, COUNT)
|
||||||
.mapToObj(i -> new Put(getRow(i)).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
|
.mapToObj(i -> new Put(getRow(i)).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
|
||||||
.collect(Collectors.toList())).get();
|
.collect(Collectors.toList())).get();
|
||||||
List<Result> results =
|
List<Result> results = table.getAll(IntStream.range(0, COUNT)
|
||||||
table
|
.mapToObj(i -> Arrays.asList(new Get(getRow(i)), new Get(Arrays.copyOf(getRow(i), 4))))
|
||||||
.getAll(IntStream.range(0, COUNT)
|
.flatMap(l -> l.stream()).collect(Collectors.toList())).get();
|
||||||
.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());
|
assertEquals(2 * COUNT, results.size());
|
||||||
for (int i = 0; i < COUNT; i++) {
|
for (int i = 0; i < COUNT; i++) {
|
||||||
assertEquals(i, Bytes.toInt(results.get(2 * i).getValue(FAMILY, CQ)));
|
assertEquals(i, Bytes.toInt(results.get(2 * i).getValue(FAMILY, CQ)));
|
||||||
@ -153,19 +152,20 @@ public class TestAsyncTableBatch {
|
|||||||
}
|
}
|
||||||
Admin admin = TEST_UTIL.getAdmin();
|
Admin admin = TEST_UTIL.getAdmin();
|
||||||
admin.flush(TABLE_NAME);
|
admin.flush(TABLE_NAME);
|
||||||
TEST_UTIL.getHBaseCluster().getRegions(TABLE_NAME).forEach(r -> {
|
List<Future<?>> splitFutures =
|
||||||
byte[] startKey = r.getRegionInfo().getStartKey();
|
TEST_UTIL.getHBaseCluster().getRegions(TABLE_NAME).stream().map(r -> {
|
||||||
int number = startKey.length == 0 ? 55 : Integer.parseInt(Bytes.toString(startKey));
|
byte[] startKey = r.getRegionInfo().getStartKey();
|
||||||
byte[] splitPoint = Bytes.toBytes(String.format("%03d", number + 55));
|
int number = startKey.length == 0 ? 55 : Integer.parseInt(Bytes.toString(startKey));
|
||||||
try {
|
byte[] splitPoint = Bytes.toBytes(String.format("%03d", number + 55));
|
||||||
admin.splitRegion(r.getRegionInfo().getRegionName(), splitPoint);
|
try {
|
||||||
} catch (IOException e) {
|
return admin.splitRegionAsync(r.getRegionInfo().getRegionName(), splitPoint);
|
||||||
throw new UncheckedIOException(e);
|
} 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
|
}).collect(Collectors.toList());
|
||||||
// and then start our work.
|
for (Future<?> future : splitFutures) {
|
||||||
Thread.sleep(5000);
|
future.get(30, TimeUnit.SECONDS);
|
||||||
|
}
|
||||||
table.deleteAll(
|
table.deleteAll(
|
||||||
IntStream.range(0, COUNT).mapToObj(i -> new Delete(getRow(i))).collect(Collectors.toList()))
|
IntStream.range(0, COUNT).mapToObj(i -> new Delete(getRow(i))).collect(Collectors.toList()))
|
||||||
.get();
|
.get();
|
||||||
@ -179,7 +179,7 @@ public class TestAsyncTableBatch {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMixed() throws InterruptedException, ExecutionException {
|
public void testMixed() throws InterruptedException, ExecutionException {
|
||||||
AsyncTableBase table = tableGetter.apply(TABLE_NAME);
|
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
|
||||||
table.putAll(IntStream.range(0, 5)
|
table.putAll(IntStream.range(0, 5)
|
||||||
.mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(FAMILY, CQ, Bytes.toBytes((long) i)))
|
.mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(FAMILY, CQ, Bytes.toBytes((long) i)))
|
||||||
.collect(Collectors.toList())).get();
|
.collect(Collectors.toList())).get();
|
||||||
@ -223,10 +223,10 @@ public class TestAsyncTableBatch {
|
|||||||
@Test
|
@Test
|
||||||
public void testPartialSuccess() throws IOException, InterruptedException, ExecutionException {
|
public void testPartialSuccess() throws IOException, InterruptedException, ExecutionException {
|
||||||
Admin admin = TEST_UTIL.getAdmin();
|
Admin admin = TEST_UTIL.getAdmin();
|
||||||
HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLE_NAME));
|
TableDescriptor htd = TableDescriptorBuilder.newBuilder(admin.getDescriptor(TABLE_NAME))
|
||||||
htd.addCoprocessor(ErrorInjectObserver.class.getName());
|
.addCoprocessor(ErrorInjectObserver.class.getName()).build();
|
||||||
admin.modifyTable(TABLE_NAME, htd);
|
admin.modifyTable(htd);
|
||||||
AsyncTableBase table = tableGetter.apply(TABLE_NAME);
|
AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
|
||||||
table.putAll(Arrays.asList(SPLIT_KEYS).stream().map(k -> new Put(k).addColumn(FAMILY, CQ, k))
|
table.putAll(Arrays.asList(SPLIT_KEYS).stream().map(k -> new Put(k).addColumn(FAMILY, CQ, k))
|
||||||
.collect(Collectors.toList())).get();
|
.collect(Collectors.toList())).get();
|
||||||
List<CompletableFuture<Result>> futures = table
|
List<CompletableFuture<Result>> futures = table
|
||||||
|
@ -18,7 +18,7 @@
|
|||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import static org.apache.hadoop.hbase.HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT;
|
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 static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@ -38,7 +38,6 @@ import java.util.stream.IntStream;
|
|||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
|
||||||
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
|
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
@ -71,7 +70,7 @@ public class TestAsyncTableGetMultiThreaded {
|
|||||||
|
|
||||||
private static AsyncConnection CONN;
|
private static AsyncConnection CONN;
|
||||||
|
|
||||||
private static RawAsyncTable TABLE;
|
private static AsyncTable<?> TABLE;
|
||||||
|
|
||||||
private static byte[][] SPLIT_KEYS;
|
private static byte[][] SPLIT_KEYS;
|
||||||
|
|
||||||
@ -80,8 +79,7 @@ public class TestAsyncTableGetMultiThreaded {
|
|||||||
setUp(MemoryCompactionPolicy.NONE);
|
setUp(MemoryCompactionPolicy.NONE);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static void setUp(MemoryCompactionPolicy memoryCompaction)
|
protected static void setUp(MemoryCompactionPolicy memoryCompaction) throws Exception {
|
||||||
throws Exception {
|
|
||||||
TEST_UTIL.getConfiguration().set(TABLES_ON_MASTER, "none");
|
TEST_UTIL.getConfiguration().set(TABLES_ON_MASTER, "none");
|
||||||
TEST_UTIL.getConfiguration().setLong(HBASE_CLIENT_META_OPERATION_TIMEOUT, 60000L);
|
TEST_UTIL.getConfiguration().setLong(HBASE_CLIENT_META_OPERATION_TIMEOUT, 60000L);
|
||||||
TEST_UTIL.getConfiguration().setInt(ByteBufferPool.MAX_POOL_SIZE_KEY, 100);
|
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.createTable(TABLE_NAME, FAMILY);
|
||||||
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
||||||
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
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();
|
.setMaxRetries(1000).build();
|
||||||
TABLE.putAll(
|
TABLE.putAll(
|
||||||
IntStream.range(0, COUNT).mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
|
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 {
|
public void test() throws IOException, InterruptedException, ExecutionException {
|
||||||
int numThreads = 20;
|
int numThreads = 20;
|
||||||
AtomicBoolean stop = new AtomicBoolean(false);
|
AtomicBoolean stop = new AtomicBoolean(false);
|
||||||
ExecutorService executor = Executors.newFixedThreadPool(numThreads,
|
ExecutorService executor =
|
||||||
Threads.newDaemonThreadFactory("TestAsyncGet-"));
|
Executors.newFixedThreadPool(numThreads, Threads.newDaemonThreadFactory("TestAsyncGet-"));
|
||||||
List<Future<?>> futures = new ArrayList<>();
|
List<Future<?>> futures = new ArrayList<>();
|
||||||
IntStream.range(0, numThreads).forEach(i -> futures.add(executor.submit(() -> {
|
IntStream.range(0, numThreads).forEach(i -> futures.add(executor.submit(() -> {
|
||||||
run(stop);
|
run(stop);
|
||||||
@ -138,13 +136,13 @@ public class TestAsyncTableGetMultiThreaded {
|
|||||||
region.compact(true);
|
region.compact(true);
|
||||||
}
|
}
|
||||||
Thread.sleep(5000);
|
Thread.sleep(5000);
|
||||||
admin.balancer(true);
|
admin.balance(true);
|
||||||
Thread.sleep(5000);
|
Thread.sleep(5000);
|
||||||
ServerName metaServer = TEST_UTIL.getHBaseCluster().getServerHoldingMeta();
|
ServerName metaServer = TEST_UTIL.getHBaseCluster().getServerHoldingMeta();
|
||||||
ServerName newMetaServer = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
|
ServerName newMetaServer = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
|
||||||
.map(t -> t.getRegionServer().getServerName()).filter(s -> !s.equals(metaServer))
|
.map(t -> t.getRegionServer().getServerName()).filter(s -> !s.equals(metaServer))
|
||||||
.findAny().get();
|
.findAny().get();
|
||||||
admin.move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
|
admin.move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
|
||||||
Bytes.toBytes(newMetaServer.getServerName()));
|
Bytes.toBytes(newMetaServer.getServerName()));
|
||||||
Thread.sleep(5000);
|
Thread.sleep(5000);
|
||||||
}
|
}
|
||||||
|
@ -102,7 +102,7 @@ public class TestAsyncTableNoncedRetry {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAppend() throws InterruptedException, ExecutionException {
|
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();
|
Result result = table.append(new Append(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
||||||
assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER));
|
assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER));
|
||||||
result = table.append(new Append(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
result = table.append(new Append(row).addColumn(FAMILY, QUALIFIER, VALUE)).get();
|
||||||
@ -114,7 +114,7 @@ public class TestAsyncTableNoncedRetry {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIncrement() throws InterruptedException, ExecutionException {
|
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());
|
assertEquals(1L, table.incrementColumnValue(row, FAMILY, QUALIFIER, 1L).get().longValue());
|
||||||
// the second call should have no effect as we always generate the same nonce.
|
// the second call should have no effect as we always generate the same nonce.
|
||||||
assertEquals(1L, table.incrementColumnValue(row, FAMILY, QUALIFIER, 1L).get().longValue());
|
assertEquals(1L, table.incrementColumnValue(row, FAMILY, QUALIFIER, 1L).get().longValue());
|
||||||
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.ForkJoinPool;
|
import java.util.concurrent.ForkJoinPool;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
@ -42,8 +41,7 @@ public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
|
|||||||
|
|
||||||
@Parameters(name = "{index}: scan={0}")
|
@Parameters(name = "{index}: scan={0}")
|
||||||
public static List<Object[]> params() {
|
public static List<Object[]> params() {
|
||||||
return getScanCreater().stream().map(p -> new Object[] { p.getFirst(), p.getSecond() })
|
return getScanCreatorParams();
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -53,7 +51,8 @@ public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<Result> doScan(Scan scan) throws Exception {
|
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();
|
SimpleScanResultConsumer consumer = new SimpleScanResultConsumer();
|
||||||
table.scan(scan, consumer);
|
table.scan(scan, consumer);
|
||||||
List<Result> results = consumer.getAll();
|
List<Result> results = consumer.getAll();
|
||||||
|
@ -17,11 +17,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.ForkJoinPool;
|
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
@ -39,7 +36,7 @@ public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
|
|||||||
public String tableType;
|
public String tableType;
|
||||||
|
|
||||||
@Parameter(1)
|
@Parameter(1)
|
||||||
public Supplier<AsyncTableBase> getTable;
|
public Supplier<AsyncTable<?>> getTable;
|
||||||
|
|
||||||
@Parameter(2)
|
@Parameter(2)
|
||||||
public String scanType;
|
public String scanType;
|
||||||
@ -47,22 +44,9 @@ public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
|
|||||||
@Parameter(3)
|
@Parameter(3)
|
||||||
public Supplier<Scan> scanCreator;
|
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}")
|
@Parameters(name = "{index}: table={0}, scan={2}")
|
||||||
public static List<Object[]> params() {
|
public static List<Object[]> params() {
|
||||||
Supplier<AsyncTableBase> rawTable = TestAsyncTableScanAll::getRawTable;
|
return getTableAndScanCreatorParams();
|
||||||
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());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -106,8 +106,8 @@ public class TestAsyncTableScanMetrics {
|
|||||||
|
|
||||||
private static Pair<List<Result>, ScanMetrics> doScanWithRawAsyncTable(Scan scan)
|
private static Pair<List<Result>, ScanMetrics> doScanWithRawAsyncTable(Scan scan)
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
SimpleRawScanResultConsumer consumer = new SimpleRawScanResultConsumer();
|
BufferingScanResultConsumer consumer = new BufferingScanResultConsumer();
|
||||||
CONN.getRawTable(TABLE_NAME).scan(scan, consumer);
|
CONN.getTable(TABLE_NAME).scan(scan, consumer);
|
||||||
List<Result> results = new ArrayList<>();
|
List<Result> results = new ArrayList<>();
|
||||||
for (Result result; (result = consumer.take()) != null;) {
|
for (Result result; (result = consumer.take()) != null;) {
|
||||||
results.add(result);
|
results.add(result);
|
||||||
|
@ -49,7 +49,7 @@ public class TestAsyncTableScanRenewLease {
|
|||||||
|
|
||||||
private static AsyncConnection CONN;
|
private static AsyncConnection CONN;
|
||||||
|
|
||||||
private static RawAsyncTable TABLE;
|
private static AsyncTable<AdvancedScanResultConsumer> TABLE;
|
||||||
|
|
||||||
private static int SCANNER_LEASE_TIMEOUT_PERIOD_MS = 5000;
|
private static int SCANNER_LEASE_TIMEOUT_PERIOD_MS = 5000;
|
||||||
|
|
||||||
@ -60,7 +60,7 @@ public class TestAsyncTableScanRenewLease {
|
|||||||
TEST_UTIL.startMiniCluster(1);
|
TEST_UTIL.startMiniCluster(1);
|
||||||
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
|
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
|
||||||
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
||||||
TABLE = CONN.getRawTable(TABLE_NAME);
|
TABLE = CONN.getTable(TABLE_NAME);
|
||||||
TABLE.putAll(IntStream.range(0, 10).mapToObj(
|
TABLE.putAll(IntStream.range(0, 10).mapToObj(
|
||||||
i -> new Put(Bytes.toBytes(String.format("%02d", i))).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
|
i -> new Put(Bytes.toBytes(String.format("%02d", i))).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
|
||||||
.collect(Collectors.toList())).get();
|
.collect(Collectors.toList())).get();
|
||||||
@ -72,7 +72,7 @@ public class TestAsyncTableScanRenewLease {
|
|||||||
TEST_UTIL.shutdownMiniCluster();
|
TEST_UTIL.shutdownMiniCluster();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final class RenewLeaseConsumer implements RawScanResultConsumer {
|
private static final class RenewLeaseConsumer implements AdvancedScanResultConsumer {
|
||||||
|
|
||||||
private final List<Result> results = new ArrayList<>();
|
private final List<Result> results = new ArrayList<>();
|
||||||
|
|
||||||
|
@ -21,7 +21,6 @@ import java.util.ArrayList;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.ForkJoinPool;
|
import java.util.concurrent.ForkJoinPool;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
@ -36,25 +35,30 @@ import org.junit.runners.Parameterized.Parameters;
|
|||||||
public class TestAsyncTableScanner extends AbstractTestAsyncTableScan {
|
public class TestAsyncTableScanner extends AbstractTestAsyncTableScan {
|
||||||
|
|
||||||
@Parameter(0)
|
@Parameter(0)
|
||||||
public String scanType;
|
public String tableType;
|
||||||
|
|
||||||
@Parameter(1)
|
@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() {
|
public static List<Object[]> params() {
|
||||||
return getScanCreater().stream().map(p -> new Object[] { p.getFirst(), p.getSecond() })
|
return getTableAndScanCreatorParams();
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Scan createScan() {
|
protected Scan createScan() {
|
||||||
return scanCreater.get();
|
return scanCreator.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<Result> doScan(Scan scan) throws Exception {
|
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<>();
|
List<Result> results = new ArrayList<>();
|
||||||
try (ResultScanner scanner = table.getScanner(scan)) {
|
try (ResultScanner scanner = table.getScanner(scan)) {
|
||||||
for (Result result; (result = scanner.next()) != null;) {
|
for (Result result; (result = scanner.next()) != null;) {
|
||||||
|
@ -47,7 +47,7 @@ public class TestAsyncTableScannerCloseWhileSuspending {
|
|||||||
|
|
||||||
private static AsyncConnection CONN;
|
private static AsyncConnection CONN;
|
||||||
|
|
||||||
private static AsyncTable TABLE;
|
private static AsyncTable<?> TABLE;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUp() throws Exception {
|
public static void setUp() throws Exception {
|
||||||
|
@ -52,9 +52,9 @@ public class TestRawAsyncScanCursor extends AbstractTestScanCursor {
|
|||||||
private void doTest(boolean reversed)
|
private void doTest(boolean reversed)
|
||||||
throws InterruptedException, ExecutionException, IOException {
|
throws InterruptedException, ExecutionException, IOException {
|
||||||
CompletableFuture<Void> future = new CompletableFuture<>();
|
CompletableFuture<Void> future = new CompletableFuture<>();
|
||||||
RawAsyncTable table = CONN.getRawTable(TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> table = CONN.getTable(TABLE_NAME);
|
||||||
table.scan(reversed ? createReversedScanWithSparseFilter() : createScanWithSparseFilter(),
|
table.scan(reversed ? createReversedScanWithSparseFilter() : createScanWithSparseFilter(),
|
||||||
new RawScanResultConsumer() {
|
new AdvancedScanResultConsumer() {
|
||||||
|
|
||||||
private int count;
|
private int count;
|
||||||
|
|
||||||
@ -121,8 +121,8 @@ public class TestRawAsyncScanCursor extends AbstractTestScanCursor {
|
|||||||
@Test
|
@Test
|
||||||
public void testSizeLimit() throws InterruptedException, ExecutionException {
|
public void testSizeLimit() throws InterruptedException, ExecutionException {
|
||||||
CompletableFuture<Void> future = new CompletableFuture<>();
|
CompletableFuture<Void> future = new CompletableFuture<>();
|
||||||
RawAsyncTable table = CONN.getRawTable(TABLE_NAME);
|
AsyncTable<AdvancedScanResultConsumer> table = CONN.getTable(TABLE_NAME);
|
||||||
table.scan(createScanWithSizeLimit(), new RawScanResultConsumer() {
|
table.scan(createScanWithSizeLimit(), new AdvancedScanResultConsumer() {
|
||||||
|
|
||||||
private int count;
|
private int count;
|
||||||
|
|
||||||
|
@ -58,14 +58,14 @@ public class TestRawAsyncTableLimitedScanWithFilter {
|
|||||||
|
|
||||||
private static AsyncConnection CONN;
|
private static AsyncConnection CONN;
|
||||||
|
|
||||||
private static RawAsyncTable TABLE;
|
private static AsyncTable<?> TABLE;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUp() throws Exception {
|
public static void setUp() throws Exception {
|
||||||
UTIL.startMiniCluster(1);
|
UTIL.startMiniCluster(1);
|
||||||
UTIL.createTable(TABLE_NAME, FAMILY);
|
UTIL.createTable(TABLE_NAME, FAMILY);
|
||||||
CONN = ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get();
|
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 -> {
|
TABLE.putAll(IntStream.range(0, ROW_COUNT).mapToObj(i -> {
|
||||||
Put put = new Put(Bytes.toBytes(i));
|
Put put = new Put(Bytes.toBytes(i));
|
||||||
IntStream.range(0, CQS.length)
|
IntStream.range(0, CQS.length)
|
||||||
|
@ -44,13 +44,13 @@ public class TestRawAsyncTablePartialScan {
|
|||||||
private static byte[] FAMILY = Bytes.toBytes("cf");
|
private static byte[] FAMILY = Bytes.toBytes("cf");
|
||||||
|
|
||||||
private static byte[][] CQS =
|
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 int COUNT = 100;
|
||||||
|
|
||||||
private static AsyncConnection CONN;
|
private static AsyncConnection CONN;
|
||||||
|
|
||||||
private static RawAsyncTable TABLE;
|
private static AsyncTable<?> TABLE;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUp() throws Exception {
|
public static void setUp() throws Exception {
|
||||||
@ -58,7 +58,7 @@ public class TestRawAsyncTablePartialScan {
|
|||||||
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
|
TEST_UTIL.createTable(TABLE_NAME, FAMILY);
|
||||||
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
TEST_UTIL.waitTableAvailable(TABLE_NAME);
|
||||||
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
||||||
TABLE = CONN.getRawTable(TABLE_NAME);
|
TABLE = CONN.getTable(TABLE_NAME);
|
||||||
TABLE
|
TABLE
|
||||||
.putAll(IntStream.range(0, COUNT)
|
.putAll(IntStream.range(0, COUNT)
|
||||||
.mapToObj(i -> new Put(Bytes.toBytes(String.format("%02d", i)))
|
.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
|
// 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.
|
// but we should get 2 + 1 for every row.
|
||||||
List<Result> results =
|
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());
|
assertEquals(2 * COUNT, results.size());
|
||||||
for (int i = 0; i < COUNT; i++) {
|
for (int i = 0; i < COUNT; i++) {
|
||||||
int row = COUNT - i - 1;
|
int row = COUNT - i - 1;
|
||||||
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
@ -42,8 +41,7 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
|
|||||||
|
|
||||||
@Parameters(name = "{index}: type={0}")
|
@Parameters(name = "{index}: type={0}")
|
||||||
public static List<Object[]> params() {
|
public static List<Object[]> params() {
|
||||||
return getScanCreater().stream().map(p -> new Object[] { p.getFirst(), p.getSecond() })
|
return getScanCreatorParams();
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -53,8 +51,8 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<Result> doScan(Scan scan) throws Exception {
|
protected List<Result> doScan(Scan scan) throws Exception {
|
||||||
SimpleRawScanResultConsumer scanConsumer = new SimpleRawScanResultConsumer();
|
BufferingScanResultConsumer scanConsumer = new BufferingScanResultConsumer();
|
||||||
ASYNC_CONN.getRawTable(TABLE_NAME).scan(scan, scanConsumer);
|
ASYNC_CONN.getTable(TABLE_NAME).scan(scan, scanConsumer);
|
||||||
List<Result> results = new ArrayList<>();
|
List<Result> results = new ArrayList<>();
|
||||||
for (Result result; (result = scanConsumer.take()) != null;) {
|
for (Result result; (result = scanConsumer.take()) != null;) {
|
||||||
results.add(result);
|
results.add(result);
|
||||||
|
Loading…
x
Reference in New Issue
Block a user