HBASE-13071 Hbase Streaming Scan Feature
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
220ac141bf
commit
86b91997d0
|
@ -0,0 +1,234 @@
|
||||||
|
/**
|
||||||
|
* 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.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Queue;
|
||||||
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* ClientAsyncPrefetchScanner implements async scanner behaviour.
|
||||||
|
* Specifically, the cache used by this scanner is a concurrent queue which allows both
|
||||||
|
* the producer (hbase client) and consumer (application) to access the queue in parallel.
|
||||||
|
* The number of rows returned in a prefetch is defined by the caching factor and the result size
|
||||||
|
* factor.
|
||||||
|
* This class allocates a buffer cache, whose size is a function of both factors.
|
||||||
|
* The prefetch is invoked when the cache is halffilled, instead of waiting for it to be empty.
|
||||||
|
* This is defined in the method {@link ClientAsyncPrefetchScanner#prefetchCondition()}.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class ClientAsyncPrefetchScanner extends ClientScanner {
|
||||||
|
|
||||||
|
private static final int ESTIMATED_SINGLE_RESULT_SIZE = 1024;
|
||||||
|
private static final int DEFAULT_QUEUE_CAPACITY = 1024;
|
||||||
|
|
||||||
|
private int cacheCapacity;
|
||||||
|
private AtomicLong cacheSizeInBytes;
|
||||||
|
// exception queue (from prefetch to main scan execution)
|
||||||
|
private Queue<Exception> exceptionsQueue;
|
||||||
|
// prefetch runnable object to be executed asynchronously
|
||||||
|
private PrefetchRunnable prefetchRunnable;
|
||||||
|
// Boolean flag to ensure only a single prefetch is running (per scan)
|
||||||
|
// We use atomic boolean to allow multiple concurrent threads to
|
||||||
|
// consume records from the same cache, but still have a single prefetcher thread.
|
||||||
|
// For a single consumer thread this can be replace with a native boolean.
|
||||||
|
private AtomicBoolean prefetchRunning;
|
||||||
|
// an attribute for synchronizing close between scanner and prefetch threads
|
||||||
|
private AtomicLong closingThreadId;
|
||||||
|
private static final int NO_THREAD = -1;
|
||||||
|
|
||||||
|
public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name,
|
||||||
|
ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
|
||||||
|
RpcControllerFactory rpcControllerFactory, ExecutorService pool,
|
||||||
|
int replicaCallTimeoutMicroSecondScan) throws IOException {
|
||||||
|
super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool,
|
||||||
|
replicaCallTimeoutMicroSecondScan);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void initCache() {
|
||||||
|
// concurrent cache
|
||||||
|
cacheCapacity = calcCacheCapacity();
|
||||||
|
cache = new LinkedBlockingQueue<Result>(cacheCapacity);
|
||||||
|
cacheSizeInBytes = new AtomicLong(0);
|
||||||
|
exceptionsQueue = new ConcurrentLinkedQueue<Exception>();
|
||||||
|
prefetchRunnable = new PrefetchRunnable();
|
||||||
|
prefetchRunning = new AtomicBoolean(false);
|
||||||
|
closingThreadId = new AtomicLong(NO_THREAD);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Result next() throws IOException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
handleException();
|
||||||
|
|
||||||
|
// If the scanner is closed and there's nothing left in the cache, next is a no-op.
|
||||||
|
if (getCacheCount() == 0 && this.closed) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
if (prefetchCondition()) {
|
||||||
|
// run prefetch in the background only if no prefetch is already running
|
||||||
|
if (!isPrefetchRunning()) {
|
||||||
|
if (prefetchRunning.compareAndSet(false, true)) {
|
||||||
|
getPool().execute(prefetchRunnable);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
while (isPrefetchRunning()) {
|
||||||
|
// prefetch running or still pending
|
||||||
|
if (getCacheCount() > 0) {
|
||||||
|
Result res = cache.poll();
|
||||||
|
long estimatedSize = calcEstimatedSize(res);
|
||||||
|
addEstimatedSize(-estimatedSize);
|
||||||
|
return res;
|
||||||
|
} else {
|
||||||
|
// (busy) wait for a record - sleep
|
||||||
|
Threads.sleep(1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (getCacheCount() > 0) {
|
||||||
|
return cache.poll();
|
||||||
|
}
|
||||||
|
|
||||||
|
// if we exhausted this scanner before calling close, write out the scan metrics
|
||||||
|
writeScanMetrics();
|
||||||
|
return null;
|
||||||
|
} finally {
|
||||||
|
handleException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
if (!scanMetricsPublished) writeScanMetrics();
|
||||||
|
closed = true;
|
||||||
|
if (!isPrefetchRunning()) {
|
||||||
|
if(closingThreadId.compareAndSet(NO_THREAD, Thread.currentThread().getId())) {
|
||||||
|
super.close();
|
||||||
|
}
|
||||||
|
} // else do nothing since the async prefetch still needs this resources
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCacheCount() {
|
||||||
|
if(cache != null) {
|
||||||
|
int size = cache.size();
|
||||||
|
if(size > cacheCapacity) {
|
||||||
|
cacheCapacity = size;
|
||||||
|
}
|
||||||
|
return size;
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void addEstimatedSize(long estimatedSize) {
|
||||||
|
cacheSizeInBytes.addAndGet(estimatedSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void handleException() throws IOException {
|
||||||
|
//The prefetch task running in the background puts any exception it
|
||||||
|
//catches into this exception queue.
|
||||||
|
// Rethrow the exception so the application can handle it.
|
||||||
|
while (!exceptionsQueue.isEmpty()) {
|
||||||
|
Exception first = exceptionsQueue.peek();
|
||||||
|
first.printStackTrace();
|
||||||
|
if (first instanceof IOException) {
|
||||||
|
throw (IOException) first;
|
||||||
|
}
|
||||||
|
throw (RuntimeException) first;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isPrefetchRunning() {
|
||||||
|
return prefetchRunning.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
// double buffer - double cache size
|
||||||
|
private int calcCacheCapacity() {
|
||||||
|
int capacity = Integer.MAX_VALUE;
|
||||||
|
if(caching > 0 && caching < (Integer.MAX_VALUE /2)) {
|
||||||
|
capacity = caching * 2 + 1;
|
||||||
|
}
|
||||||
|
if(capacity == Integer.MAX_VALUE){
|
||||||
|
if(maxScannerResultSize != Integer.MAX_VALUE) {
|
||||||
|
capacity = (int) (maxScannerResultSize / ESTIMATED_SINGLE_RESULT_SIZE);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
capacity = DEFAULT_QUEUE_CAPACITY;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return capacity;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean prefetchCondition() {
|
||||||
|
return
|
||||||
|
(getCacheCount() < getCountThreshold()) &&
|
||||||
|
(maxScannerResultSize == Long.MAX_VALUE ||
|
||||||
|
getCacheSizeInBytes() < getSizeThreshold()) ;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getCountThreshold() {
|
||||||
|
return cacheCapacity / 2 ;
|
||||||
|
}
|
||||||
|
|
||||||
|
private long getSizeThreshold() {
|
||||||
|
return maxScannerResultSize / 2 ;
|
||||||
|
}
|
||||||
|
|
||||||
|
private long getCacheSizeInBytes() {
|
||||||
|
return cacheSizeInBytes.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private class PrefetchRunnable implements Runnable {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
loadCache();
|
||||||
|
} catch (Exception e) {
|
||||||
|
exceptionsQueue.add(e);
|
||||||
|
} finally {
|
||||||
|
prefetchRunning.set(false);
|
||||||
|
if(closed) {
|
||||||
|
if (closingThreadId.compareAndSet(NO_THREAD, Thread.currentThread().getId())) {
|
||||||
|
// close was waiting for the prefetch to end
|
||||||
|
close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -17,14 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import java.io.IOException;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import java.io.InterruptedIOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.LinkedList;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
|
|
||||||
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.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -45,7 +38,14 @@ import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import java.io.IOException;
|
||||||
|
import java.io.InterruptedIOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Queue;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implements the scanner interface for the HBase client.
|
* Implements the scanner interface for the HBase client.
|
||||||
|
@ -53,7 +53,7 @@ import com.google.common.annotations.VisibleForTesting;
|
||||||
* through them all.
|
* through them all.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ClientScanner extends AbstractClientScanner {
|
public abstract class ClientScanner extends AbstractClientScanner {
|
||||||
private static final Log LOG = LogFactory.getLog(ClientScanner.class);
|
private static final Log LOG = LogFactory.getLog(ClientScanner.class);
|
||||||
// A byte array in which all elements are the max byte, and it is used to
|
// A byte array in which all elements are the max byte, and it is used to
|
||||||
// construct closest front row
|
// construct closest front row
|
||||||
|
@ -64,7 +64,7 @@ public class ClientScanner extends AbstractClientScanner {
|
||||||
// wonky: e.g. if it splits on us.
|
// wonky: e.g. if it splits on us.
|
||||||
protected HRegionInfo currentRegion = null;
|
protected HRegionInfo currentRegion = null;
|
||||||
protected ScannerCallableWithReplicas callable = null;
|
protected ScannerCallableWithReplicas callable = null;
|
||||||
protected final LinkedList<Result> cache = new LinkedList<Result>();
|
protected Queue<Result> cache;
|
||||||
/**
|
/**
|
||||||
* A list of partial results that have been returned from the server. This list should only
|
* A list of partial results that have been returned from the server. This list should only
|
||||||
* contain results if this scanner does not have enough partial results to form the complete
|
* contain results if this scanner does not have enough partial results to form the complete
|
||||||
|
@ -151,9 +151,12 @@ public class ClientScanner extends AbstractClientScanner {
|
||||||
this.rpcControllerFactory = controllerFactory;
|
this.rpcControllerFactory = controllerFactory;
|
||||||
|
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
|
initCache();
|
||||||
initializeScannerInConstruction();
|
initializeScannerInConstruction();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected abstract void initCache();
|
||||||
|
|
||||||
protected void initializeScannerInConstruction() throws IOException{
|
protected void initializeScannerInConstruction() throws IOException{
|
||||||
// initialize the scanner
|
// initialize the scanner
|
||||||
nextScanner(this.caching, false);
|
nextScanner(this.caching, false);
|
||||||
|
@ -342,8 +345,11 @@ public class ClientScanner extends AbstractClientScanner {
|
||||||
scanMetricsPublished = true;
|
scanMetricsPublished = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
protected void initSyncCache() {
|
||||||
public Result next() throws IOException {
|
cache = new LinkedList<Result>();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Result nextWithSyncCache() throws IOException {
|
||||||
// If the scanner is closed and there's nothing left in the cache, next is a no-op.
|
// If the scanner is closed and there's nothing left in the cache, next is a no-op.
|
||||||
if (cache.size() == 0 && this.closed) {
|
if (cache.size() == 0 && this.closed) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -370,6 +376,8 @@ public class ClientScanner extends AbstractClientScanner {
|
||||||
* Contact the servers to load more {@link Result}s in the cache.
|
* Contact the servers to load more {@link Result}s in the cache.
|
||||||
*/
|
*/
|
||||||
protected void loadCache() throws IOException {
|
protected void loadCache() throws IOException {
|
||||||
|
// check if scanner was closed during previous prefetch
|
||||||
|
if (closed) return;
|
||||||
Result[] values = null;
|
Result[] values = null;
|
||||||
long remainingResultSize = maxScannerResultSize;
|
long remainingResultSize = maxScannerResultSize;
|
||||||
int countdown = this.caching;
|
int countdown = this.caching;
|
||||||
|
@ -482,11 +490,10 @@ public class ClientScanner extends AbstractClientScanner {
|
||||||
if (!resultsToAddToCache.isEmpty()) {
|
if (!resultsToAddToCache.isEmpty()) {
|
||||||
for (Result rs : resultsToAddToCache) {
|
for (Result rs : resultsToAddToCache) {
|
||||||
cache.add(rs);
|
cache.add(rs);
|
||||||
// We don't make Iterator here
|
long estimatedHeapSizeOfResult = calcEstimatedSize(rs);
|
||||||
for (Cell cell : rs.rawCells()) {
|
|
||||||
remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
|
|
||||||
}
|
|
||||||
countdown--;
|
countdown--;
|
||||||
|
remainingResultSize -= estimatedHeapSizeOfResult;
|
||||||
|
addEstimatedSize(estimatedHeapSizeOfResult);
|
||||||
this.lastResult = rs;
|
this.lastResult = rs;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -532,6 +539,24 @@ public class ClientScanner extends AbstractClientScanner {
|
||||||
return remainingResultSize > 0 && remainingRows > 0 && !regionHasMoreResults;
|
return remainingResultSize > 0 && remainingRows > 0 && !regionHasMoreResults;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected long calcEstimatedSize(Result rs) {
|
||||||
|
long estimatedHeapSizeOfResult = 0;
|
||||||
|
// We don't make Iterator here
|
||||||
|
for (Cell cell : rs.rawCells()) {
|
||||||
|
estimatedHeapSizeOfResult += CellUtil.estimatedHeapSizeOf(cell);
|
||||||
|
}
|
||||||
|
return estimatedHeapSizeOfResult;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void addEstimatedSize(long estimatedHeapSizeOfResult) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public int getCacheCount() {
|
||||||
|
return cache != null ? cache.size() : 0;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method ensures all of our book keeping regarding partial results is kept up to date. This
|
* This method ensures all of our book keeping regarding partial results is kept up to date. This
|
||||||
* method should be called once we know that the results we received back from the RPC request do
|
* method should be called once we know that the results we received back from the RPC request do
|
||||||
|
|
|
@ -0,0 +1,54 @@
|
||||||
|
/**
|
||||||
|
* 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.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* ClientSimpleScanner implements a sync scanner behaviour.
|
||||||
|
* The cache is a simple list.
|
||||||
|
* The prefetch is invoked only when the application finished processing the entire cache.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class ClientSimpleScanner extends ClientScanner {
|
||||||
|
|
||||||
|
public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name,
|
||||||
|
ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
|
||||||
|
RpcControllerFactory rpcControllerFactory, ExecutorService pool,
|
||||||
|
int replicaCallTimeoutMicroSecondScan) throws IOException {
|
||||||
|
super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool,
|
||||||
|
replicaCallTimeoutMicroSecondScan);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void initCache() {
|
||||||
|
initSyncCache();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Result next() throws IOException {
|
||||||
|
return nextWithSyncCache();
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,10 +18,8 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import java.io.IOException;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import java.io.InterruptedIOException;
|
import com.google.protobuf.ServiceException;
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
|
|
||||||
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.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -39,8 +37,9 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import java.io.IOException;
|
||||||
import com.google.protobuf.ServiceException;
|
import java.io.InterruptedIOException;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Client scanner for small scan. Generally, only one RPC is called to fetch the
|
* Client scanner for small scan. Generally, only one RPC is called to fetch the
|
||||||
|
@ -50,7 +49,7 @@ import com.google.protobuf.ServiceException;
|
||||||
* For small scan, it will get better performance than {@link ClientScanner}
|
* For small scan, it will get better performance than {@link ClientScanner}
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ClientSmallScanner extends ClientScanner {
|
public class ClientSmallScanner extends ClientSimpleScanner {
|
||||||
private static final Log LOG = LogFactory.getLog(ClientSmallScanner.class);
|
private static final Log LOG = LogFactory.getLog(ClientSmallScanner.class);
|
||||||
private ScannerCallableWithReplicas smallScanCallable = null;
|
private ScannerCallableWithReplicas smallScanCallable = null;
|
||||||
private SmallScannerCallableFactory callableFactory;
|
private SmallScannerCallableFactory callableFactory;
|
||||||
|
|
|
@ -620,6 +620,11 @@ public class HTable implements HTableInterface {
|
||||||
scan.setMaxResultSize(scannerMaxResultSize);
|
scan.setMaxResultSize(scannerMaxResultSize);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Boolean async = scan.isAsyncPrefetch();
|
||||||
|
if (async == null) {
|
||||||
|
async = tableConfiguration.isClientScannerAsyncPrefetch();
|
||||||
|
}
|
||||||
|
|
||||||
if (scan.isReversed()) {
|
if (scan.isReversed()) {
|
||||||
if (scan.isSmall()) {
|
if (scan.isSmall()) {
|
||||||
return new ClientSmallReversedScanner(getConfiguration(), scan, getName(),
|
return new ClientSmallReversedScanner(getConfiguration(), scan, getName(),
|
||||||
|
@ -637,9 +642,15 @@ public class HTable implements HTableInterface {
|
||||||
this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
|
this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
|
||||||
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||||
} else {
|
} else {
|
||||||
return new ClientScanner(getConfiguration(), scan, getName(), this.connection,
|
if (async) {
|
||||||
this.rpcCallerFactory, this.rpcControllerFactory,
|
return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), this.connection,
|
||||||
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
this.rpcCallerFactory, this.rpcControllerFactory,
|
||||||
|
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||||
|
} else {
|
||||||
|
return new ClientSimpleScanner(getConfiguration(), scan, getName(), this.connection,
|
||||||
|
this.rpcCallerFactory, this.rpcControllerFactory,
|
||||||
|
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||||
* A reversed client scanner which support backward scanning
|
* A reversed client scanner which support backward scanning
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class ReversedClientScanner extends ClientScanner {
|
public class ReversedClientScanner extends ClientSimpleScanner {
|
||||||
private static final Log LOG = LogFactory.getLog(ReversedClientScanner.class);
|
private static final Log LOG = LogFactory.getLog(ReversedClientScanner.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -145,8 +145,23 @@ public class Scan extends Query {
|
||||||
private Map<byte [], NavigableSet<byte []>> familyMap =
|
private Map<byte [], NavigableSet<byte []>> familyMap =
|
||||||
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
|
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
|
||||||
private Boolean loadColumnFamiliesOnDemand = null;
|
private Boolean loadColumnFamiliesOnDemand = null;
|
||||||
|
private Boolean asyncPrefetch = null;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Parameter name for client scanner sync/async prefetch toggle.
|
||||||
|
* When using async scanner, prefetching data from the server is done at the background.
|
||||||
|
* The parameter currently won't have any effect in the case that the user has set
|
||||||
|
* Scan#setSmall or Scan#setReversed
|
||||||
|
*/
|
||||||
|
public static final String HBASE_CLIENT_SCANNER_ASYNC_PREFETCH =
|
||||||
|
"hbase.client.scanner.async.prefetch";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default value of {@link #HBASE_CLIENT_SCANNER_ASYNC_PREFETCH}.
|
||||||
|
*/
|
||||||
|
public static final boolean DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH = false;
|
||||||
|
|
||||||
|
/**
|
||||||
* Set it true for small scan to get better performance
|
* Set it true for small scan to get better performance
|
||||||
*
|
*
|
||||||
* Small scan should use pread and big scan can use seek + read
|
* Small scan should use pread and big scan can use seek + read
|
||||||
|
@ -255,6 +270,7 @@ public class Scan extends Query {
|
||||||
this.tr = get.getTimeRange();
|
this.tr = get.getTimeRange();
|
||||||
this.familyMap = get.getFamilyMap();
|
this.familyMap = get.getFamilyMap();
|
||||||
this.getScan = true;
|
this.getScan = true;
|
||||||
|
this.asyncPrefetch = false;
|
||||||
this.consistency = get.getConsistency();
|
this.consistency = get.getConsistency();
|
||||||
for (Map.Entry<String, byte[]> attr : get.getAttributesMap().entrySet()) {
|
for (Map.Entry<String, byte[]> attr : get.getAttributesMap().entrySet()) {
|
||||||
setAttribute(attr.getKey(), attr.getValue());
|
setAttribute(attr.getKey(), attr.getValue());
|
||||||
|
@ -971,4 +987,16 @@ public class Scan extends Query {
|
||||||
if (bytes == null) return null;
|
if (bytes == null) return null;
|
||||||
return ProtobufUtil.toScanMetrics(bytes);
|
return ProtobufUtil.toScanMetrics(bytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public Boolean isAsyncPrefetch() {
|
||||||
|
return asyncPrefetch;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Scan setAsyncPrefetch(boolean asyncPrefetch) {
|
||||||
|
this.asyncPrefetch = asyncPrefetch;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
|
@ -43,7 +43,10 @@ public class TableConfiguration {
|
||||||
private final int retries;
|
private final int retries;
|
||||||
private final int maxKeyValueSize;
|
private final int maxKeyValueSize;
|
||||||
|
|
||||||
/**
|
// toggle for async/sync prefetch
|
||||||
|
private final boolean clientScannerAsyncPrefetch;
|
||||||
|
|
||||||
|
/**
|
||||||
* Constructor
|
* Constructor
|
||||||
* @param conf Configuration object
|
* @param conf Configuration object
|
||||||
*/
|
*/
|
||||||
|
@ -73,6 +76,9 @@ public class TableConfiguration {
|
||||||
this.retries = conf.getInt(
|
this.retries = conf.getInt(
|
||||||
HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
|
HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
|
||||||
|
|
||||||
|
this.clientScannerAsyncPrefetch = conf.getBoolean(
|
||||||
|
Scan.HBASE_CLIENT_SCANNER_ASYNC_PREFETCH, Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH);
|
||||||
|
|
||||||
this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT);
|
this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -91,6 +97,7 @@ public class TableConfiguration {
|
||||||
this.primaryCallTimeoutMicroSecond = 10000;
|
this.primaryCallTimeoutMicroSecond = 10000;
|
||||||
this.replicaCallTimeoutMicroSecondScan = 1000000;
|
this.replicaCallTimeoutMicroSecondScan = 1000000;
|
||||||
this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
|
this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
|
||||||
|
this.clientScannerAsyncPrefetch = Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH;
|
||||||
this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT;
|
this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -129,4 +136,9 @@ public class TableConfiguration {
|
||||||
public long getScannerMaxResultSize() {
|
public long getScannerMaxResultSize() {
|
||||||
return scannerMaxResultSize;
|
return scannerMaxResultSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean isClientScannerAsyncPrefetch() {
|
||||||
|
return clientScannerAsyncPrefetch;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.LinkedList;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
|
|
||||||
|
@ -121,6 +122,15 @@ public class TestClientScanner {
|
||||||
public void setRpcFinished(boolean rpcFinished) {
|
public void setRpcFinished(boolean rpcFinished) {
|
||||||
this.rpcFinished = rpcFinished;
|
this.rpcFinished = rpcFinished;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void initCache() {
|
||||||
|
initSyncCache();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override public Result next() throws IOException {
|
||||||
|
return nextWithSyncCache();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -17,23 +17,9 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.charset.StandardCharsets;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.Executors;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.*;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
|
import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
|
||||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||||
|
@ -46,6 +32,16 @@ import org.mockito.Mockito;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
import org.mockito.stubbing.Answer;
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.Queue;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test the ClientSmallReversedScanner.
|
* Test the ClientSmallReversedScanner.
|
||||||
*/
|
*/
|
||||||
|
@ -178,7 +174,7 @@ public class TestClientSmallReversedScanner {
|
||||||
|
|
||||||
csrs.loadCache();
|
csrs.loadCache();
|
||||||
|
|
||||||
List<Result> results = csrs.cache;
|
Queue<Result> results = csrs.cache;
|
||||||
Iterator<Result> iter = results.iterator();
|
Iterator<Result> iter = results.iterator();
|
||||||
assertEquals(3, results.size());
|
assertEquals(3, results.size());
|
||||||
for (int i = 3; i >= 1 && iter.hasNext(); i--) {
|
for (int i = 3; i >= 1 && iter.hasNext(); i--) {
|
||||||
|
@ -248,7 +244,7 @@ public class TestClientSmallReversedScanner {
|
||||||
|
|
||||||
csrs.loadCache();
|
csrs.loadCache();
|
||||||
|
|
||||||
List<Result> results = csrs.cache;
|
Queue<Result> results = csrs.cache;
|
||||||
Iterator<Result> iter = results.iterator();
|
Iterator<Result> iter = results.iterator();
|
||||||
assertEquals(2, results.size());
|
assertEquals(2, results.size());
|
||||||
for (int i = 3; i >= 2 && iter.hasNext(); i--) {
|
for (int i = 3; i >= 2 && iter.hasNext(); i--) {
|
||||||
|
@ -264,7 +260,7 @@ public class TestClientSmallReversedScanner {
|
||||||
csrs.loadCache();
|
csrs.loadCache();
|
||||||
|
|
||||||
assertEquals(1, results.size());
|
assertEquals(1, results.size());
|
||||||
Result result = results.get(0);
|
Result result = results.peek();
|
||||||
assertEquals("row1", new String(result.getRow(), StandardCharsets.UTF_8));
|
assertEquals("row1", new String(result.getRow(), StandardCharsets.UTF_8));
|
||||||
assertEquals(1, result.getMap().size());
|
assertEquals(1, result.getMap().size());
|
||||||
|
|
||||||
|
|
|
@ -17,22 +17,9 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.charset.StandardCharsets;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.Executors;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.*;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
|
import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
|
||||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||||
|
@ -45,6 +32,15 @@ import org.mockito.Mockito;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
import org.mockito.stubbing.Answer;
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.Queue;
|
||||||
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test the ClientSmallScanner.
|
* Test the ClientSmallScanner.
|
||||||
*/
|
*/
|
||||||
|
@ -176,10 +172,10 @@ public class TestClientSmallScanner {
|
||||||
|
|
||||||
css.loadCache();
|
css.loadCache();
|
||||||
|
|
||||||
List<Result> results = css.cache;
|
Queue<Result> results = css.cache;
|
||||||
assertEquals(3, results.size());
|
assertEquals(3, results.size());
|
||||||
for (int i = 1; i <= 3; i++) {
|
for (int i = 1; i <= 3; i++) {
|
||||||
Result result = results.get(i - 1);
|
Result result = results.poll();
|
||||||
byte[] row = result.getRow();
|
byte[] row = result.getRow();
|
||||||
assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
|
assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
|
||||||
assertEquals(1, result.getMap().size());
|
assertEquals(1, result.getMap().size());
|
||||||
|
@ -243,10 +239,10 @@ public class TestClientSmallScanner {
|
||||||
|
|
||||||
css.loadCache();
|
css.loadCache();
|
||||||
|
|
||||||
List<Result> results = css.cache;
|
Queue<Result> results = css.cache;
|
||||||
assertEquals(2, results.size());
|
assertEquals(2, results.size());
|
||||||
for (int i = 1; i <= 2; i++) {
|
for (int i = 1; i <= 2; i++) {
|
||||||
Result result = results.get(i - 1);
|
Result result = results.poll();
|
||||||
byte[] row = result.getRow();
|
byte[] row = result.getRow();
|
||||||
assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
|
assertEquals("row" + i, new String(row, StandardCharsets.UTF_8));
|
||||||
assertEquals(1, result.getMap().size());
|
assertEquals(1, result.getMap().size());
|
||||||
|
@ -258,7 +254,7 @@ public class TestClientSmallScanner {
|
||||||
css.loadCache();
|
css.loadCache();
|
||||||
|
|
||||||
assertEquals(1, results.size());
|
assertEquals(1, results.size());
|
||||||
Result result = results.get(0);
|
Result result = results.peek();
|
||||||
assertEquals("row3", new String(result.getRow(), StandardCharsets.UTF_8));
|
assertEquals("row3", new String(result.getRow(), StandardCharsets.UTF_8));
|
||||||
assertEquals(1, result.getMap().size());
|
assertEquals(1, result.getMap().size());
|
||||||
assertTrue(css.closed);
|
assertTrue(css.closed);
|
||||||
|
|
|
@ -642,6 +642,55 @@ public class TestScannersFromClientSide {
|
||||||
verifyResult(result, kvListExp, toLog, "Testing scan on re-opened region");
|
verifyResult(result, kvListExp, toLog, "Testing scan on re-opened region");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test from client side for async scan
|
||||||
|
*
|
||||||
|
* @throws Exception
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testAsyncScanner() throws Exception {
|
||||||
|
byte [] TABLE = Bytes.toBytes("testAsyncScan");
|
||||||
|
byte [][] ROWS = HTestConst.makeNAscii(ROW, 2);
|
||||||
|
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
|
||||||
|
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
|
||||||
|
|
||||||
|
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
|
||||||
|
|
||||||
|
Put put;
|
||||||
|
Scan scan;
|
||||||
|
Result result;
|
||||||
|
boolean toLog = true;
|
||||||
|
List<Cell> kvListExp, kvListScan;
|
||||||
|
|
||||||
|
kvListExp = new ArrayList<Cell>();
|
||||||
|
|
||||||
|
for (int r=0; r < ROWS.length; r++) {
|
||||||
|
put = new Put(ROWS[r]);
|
||||||
|
for (int c=0; c < FAMILIES.length; c++) {
|
||||||
|
for (int q=0; q < QUALIFIERS.length; q++) {
|
||||||
|
KeyValue kv = new KeyValue(ROWS[r], FAMILIES[c], QUALIFIERS[q], 1, VALUE);
|
||||||
|
put.add(kv);
|
||||||
|
kvListExp.add(kv);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
ht.put(put);
|
||||||
|
}
|
||||||
|
|
||||||
|
scan = new Scan();
|
||||||
|
scan.setAsyncPrefetch(true);
|
||||||
|
ResultScanner scanner = ht.getScanner(scan);
|
||||||
|
kvListScan = new ArrayList<Cell>();
|
||||||
|
while ((result = scanner.next()) != null) {
|
||||||
|
for (Cell kv : result.listCells()) {
|
||||||
|
kvListScan.add(kv);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
result = Result.create(kvListScan);
|
||||||
|
assertTrue("Not instance of async scanner",scanner instanceof ClientAsyncPrefetchScanner);
|
||||||
|
verifyResult(result, kvListExp, toLog, "Testing async scan");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
static void verifyResult(Result result, List<Cell> expKvList, boolean toLog,
|
static void verifyResult(Result result, List<Cell> expKvList, boolean toLog,
|
||||||
String msg) {
|
String msg) {
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue