HBASE-17723 ClientAsyncPrefetchScanner may end prematurely when the size of the cache is one
This commit is contained in:
parent
a49bc58a54
commit
0ecb678259
@ -17,6 +17,7 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize;
|
import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@ -26,6 +27,7 @@ import java.util.concurrent.ExecutorService;
|
|||||||
import java.util.concurrent.LinkedBlockingQueue;
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
@ -62,6 +64,8 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
|
|||||||
private AtomicBoolean prefetchRunning;
|
private AtomicBoolean prefetchRunning;
|
||||||
// an attribute for synchronizing close between scanner and prefetch threads
|
// an attribute for synchronizing close between scanner and prefetch threads
|
||||||
private AtomicLong closingThreadId;
|
private AtomicLong closingThreadId;
|
||||||
|
// used for testing
|
||||||
|
private Consumer<Boolean> prefetchListener;
|
||||||
private static final int NO_THREAD = -1;
|
private static final int NO_THREAD = -1;
|
||||||
|
|
||||||
public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name,
|
public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name,
|
||||||
@ -72,6 +76,11 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
|
|||||||
replicaCallTimeoutMicroSecondScan);
|
replicaCallTimeoutMicroSecondScan);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
void setPrefetchListener(Consumer<Boolean> prefetchListener) {
|
||||||
|
this.prefetchListener = prefetchListener;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void initCache() {
|
protected void initCache() {
|
||||||
// concurrent cache
|
// concurrent cache
|
||||||
@ -88,17 +97,21 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
|
|||||||
public Result next() throws IOException {
|
public Result next() throws IOException {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
boolean hasExecutedPrefetch = false;
|
||||||
|
do {
|
||||||
handleException();
|
handleException();
|
||||||
|
|
||||||
// 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 (getCacheCount() == 0 && this.closed) {
|
if (getCacheCount() == 0 && this.closed) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (prefetchCondition()) {
|
if (prefetchCondition()) {
|
||||||
// run prefetch in the background only if no prefetch is already running
|
// run prefetch in the background only if no prefetch is already running
|
||||||
if (!isPrefetchRunning()) {
|
if (!isPrefetchRunning()) {
|
||||||
if (prefetchRunning.compareAndSet(false, true)) {
|
if (prefetchRunning.compareAndSet(false, true)) {
|
||||||
getPool().execute(prefetchRunnable);
|
getPool().execute(prefetchRunnable);
|
||||||
|
hasExecutedPrefetch = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -116,6 +129,7 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
|
|||||||
if (getCacheCount() > 0) {
|
if (getCacheCount() > 0) {
|
||||||
return pollCache();
|
return pollCache();
|
||||||
}
|
}
|
||||||
|
} while (!hasExecutedPrefetch);
|
||||||
|
|
||||||
// if we exhausted this scanner before calling close, write out the scan metrics
|
// if we exhausted this scanner before calling close, write out the scan metrics
|
||||||
writeScanMetrics();
|
writeScanMetrics();
|
||||||
@ -219,11 +233,16 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
boolean succeed = false;
|
||||||
try {
|
try {
|
||||||
loadCache();
|
loadCache();
|
||||||
|
succeed = true;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
exceptionsQueue.add(e);
|
exceptionsQueue.add(e);
|
||||||
} finally {
|
} finally {
|
||||||
|
if (prefetchListener != null) {
|
||||||
|
prefetchListener.accept(succeed);
|
||||||
|
}
|
||||||
prefetchRunning.set(false);
|
prefetchRunning.set(false);
|
||||||
if(closed) {
|
if(closed) {
|
||||||
if (closingThreadId.compareAndSet(NO_THREAD, Thread.currentThread().getId())) {
|
if (closingThreadId.compareAndSet(NO_THREAD, Thread.currentThread().getId())) {
|
||||||
|
@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
@ -656,7 +657,9 @@ public class TestScannersFromClientSide {
|
|||||||
testAsyncScanner(TableName.valueOf(name.getMethodName()),
|
testAsyncScanner(TableName.valueOf(name.getMethodName()),
|
||||||
2,
|
2,
|
||||||
3,
|
3,
|
||||||
10);
|
10,
|
||||||
|
-1,
|
||||||
|
null);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -664,11 +667,28 @@ public class TestScannersFromClientSide {
|
|||||||
testAsyncScanner(TableName.valueOf(name.getMethodName()),
|
testAsyncScanner(TableName.valueOf(name.getMethodName()),
|
||||||
30000,
|
30000,
|
||||||
1,
|
1,
|
||||||
1);
|
1,
|
||||||
|
-1,
|
||||||
|
null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAsyncScannerWithoutCaching() throws Exception {
|
||||||
|
testAsyncScanner(TableName.valueOf(name.getMethodName()),
|
||||||
|
5,
|
||||||
|
1,
|
||||||
|
1,
|
||||||
|
1,
|
||||||
|
(b) -> {
|
||||||
|
try {
|
||||||
|
TimeUnit.MILLISECONDS.sleep(500);
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private void testAsyncScanner(TableName table, int rowNumber, int familyNumber,
|
private void testAsyncScanner(TableName table, int rowNumber, int familyNumber,
|
||||||
int qualifierNumber) throws Exception {
|
int qualifierNumber, int caching, Consumer<Boolean> listener) throws Exception {
|
||||||
assert rowNumber > 0;
|
assert rowNumber > 0;
|
||||||
assert familyNumber > 0;
|
assert familyNumber > 0;
|
||||||
assert qualifierNumber > 0;
|
assert qualifierNumber > 0;
|
||||||
@ -707,11 +727,18 @@ public class TestScannersFromClientSide {
|
|||||||
|
|
||||||
Scan scan = new Scan();
|
Scan scan = new Scan();
|
||||||
scan.setAsyncPrefetch(true);
|
scan.setAsyncPrefetch(true);
|
||||||
ResultScanner scanner = ht.getScanner(scan);
|
if (caching > 0) {
|
||||||
|
scan.setCaching(caching);
|
||||||
|
}
|
||||||
|
try (ResultScanner scanner = ht.getScanner(scan)) {
|
||||||
|
assertTrue("Not instance of async scanner",scanner instanceof ClientAsyncPrefetchScanner);
|
||||||
|
((ClientAsyncPrefetchScanner) scanner).setPrefetchListener(listener);
|
||||||
List<Cell> kvListScan = new ArrayList<>();
|
List<Cell> kvListScan = new ArrayList<>();
|
||||||
Result result;
|
Result result;
|
||||||
boolean first = true;
|
boolean first = true;
|
||||||
|
int actualRows = 0;
|
||||||
while ((result = scanner.next()) != null) {
|
while ((result = scanner.next()) != null) {
|
||||||
|
++actualRows;
|
||||||
// waiting for cache. see HBASE-17376
|
// waiting for cache. see HBASE-17376
|
||||||
if (first) {
|
if (first) {
|
||||||
TimeUnit.SECONDS.sleep(1);
|
TimeUnit.SECONDS.sleep(1);
|
||||||
@ -721,9 +748,12 @@ public class TestScannersFromClientSide {
|
|||||||
kvListScan.add(kv);
|
kvListScan.add(kv);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
assertEquals(rowNumber, actualRows);
|
||||||
|
// These cells may have different rows but it is ok. The Result#getRow
|
||||||
|
// isn't used in the verifyResult()
|
||||||
result = Result.create(kvListScan);
|
result = Result.create(kvListScan);
|
||||||
assertTrue("Not instance of async scanner",scanner instanceof ClientAsyncPrefetchScanner);
|
|
||||||
verifyResult(result, kvListExp, toLog, "Testing async scan");
|
verifyResult(result, kvListExp, toLog, "Testing async scan");
|
||||||
|
}
|
||||||
TEST_UTIL.deleteTable(table);
|
TEST_UTIL.deleteTable(table);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
x
Reference in New Issue
Block a user