This is an automated email from the ASF dual-hosted git repository.
chenhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push:
new eda9c4284d use READ_ENTRY_SCHEDULING_DELAY to stable
stickyReadsWithFailures (#3628)
eda9c4284d is described below
commit eda9c4284d434cdc4d94637f6257a1c6bf179745
Author: wenbingshen <[email protected]>
AuthorDate: Wed Dec 7 11:00:01 2022 +0800
use READ_ENTRY_SCHEDULING_DELAY to stable stickyReadsWithFailures (#3628)
### Motivation
I found the following flaky-test:
org.apache.bookkeeper.bookie.BookieStickyReadsTest.stickyReadsWithFailures:
https://github.com/apache/bookkeeper/actions/runs/3367374609/jobs/5584792353
```
Error: Tests run: 4, Failures: 0, Errors: 1, Skipped: 0, Time elapsed:
8.925 s <<< FAILURE! - in org.apache.bookkeeper.bookie.BookieStickyReadsTest
Error:
org.apache.bookkeeper.bookie.BookieStickyReadsTest.stickyReadsWithFailures
Time elapsed: 1.752 s <<< ERROR!
java.lang.IndexOutOfBoundsException: Index: -1, Size: 3
at
java.base/java.util.LinkedList.checkElementIndex(LinkedList.java:559)
at java.base/java.util.LinkedList.get(LinkedList.java:480)
at
org.apache.bookkeeper.test.BookKeeperClusterTestCase.serverByIndex(BookKeeperClusterTestCase.java:369)
at
org.apache.bookkeeper.bookie.BookieStickyReadsTest.stickyReadsWithFailures(BookieStickyReadsTest.java:153)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
at
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
at
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
at
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
at
java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at java.base/java.lang.Thread.run(Thread.java:829)
```
In the stickyReadsWithFailures test, the client successfully reads the
entry, but the `READ_ENTRY_REQUEST` metric does not get bonuses. After reading
the `READ_ENTRY_REQUEST`update logic, I found the metric updating after netty
channel sends the response successfully, the metric is updated through the
`ChannelFutureListener` callback, and the asynchronous update causes the above
test to fail.
```java
protected void sendResponse(StatusCode code, Object response, OpStatsLogger
statsLogger) {
......
if (channel.isActive()) {
channel.writeAndFlush(response).addListener(new
ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws
Exception {
long writeElapsedNanos =
MathUtils.elapsedNanos(writeNanos);
if (!future.isSuccess()) {
requestProcessor.getRequestStats().getChannelWriteStats()
.registerFailedEvent(writeElapsedNanos,
TimeUnit.NANOSECONDS);
} else {
requestProcessor.getRequestStats().getChannelWriteStats()
.registerSuccessfulEvent(writeElapsedNanos,
TimeUnit.NANOSECONDS);
}
if (StatusCode.EOK == code) {
statsLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(enqueueNanos),
TimeUnit.NANOSECONDS);
} else {
statsLogger.registerFailedEvent(MathUtils.elapsedNanos(enqueueNanos),
TimeUnit.NANOSECONDS);
}
}
});
}
......
}
```
### Changes
The `READ_ENTRY_SCHEDULING_DELAY` metric is processed before the Read
request is processed, which proved that bookie receives the read request from
the client and can well meet the needs of `BookieStickyReadsTest`.
This makes the `BookieStickyReadsTest` test more stabled.
---
.../bookkeeper/bookie/BookieStickyReadsTest.java | 19 ++++++++++---------
1 file changed, 10 insertions(+), 9 deletions(-)
diff --git
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java
index 698c816eb8..99bcc78845 100644
---
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java
+++
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieStickyReadsTest.java
@@ -48,7 +48,7 @@ public class BookieStickyReadsTest extends
BookKeeperClusterTestCase {
private static final int NUM_BOOKIES = 3;
- private static final String READ_ENTRY_REQUEST_METRIC =
"bookkeeper_server.READ_ENTRY_REQUEST";
+ private static final String READ_ENTRY_SCHEDULING_DELAY_METRIC =
"bookkeeper_server.READ_ENTRY_SCHEDULING_DELAY";
public BookieStickyReadsTest() {
super(NUM_BOOKIES);
@@ -64,7 +64,8 @@ public class BookieStickyReadsTest extends
BookKeeperClusterTestCase {
writeAndReadEntries(conf, 3, 3, 3);
// All bookies should have received at least some read request
- getBookieReadRequestStats().values().forEach(readRequests ->
assertTrue(readRequests > 0));
+ getBookieReadEntrySchedulingDelayStats().values()
+ .forEach(readRequests -> assertTrue(readRequests > 0));
}
@Test
@@ -76,7 +77,8 @@ public class BookieStickyReadsTest extends
BookKeeperClusterTestCase {
// All bookies should have received at least some read request since we
// don't enable sticky reads when striping is enabled
- getBookieReadRequestStats().values().forEach(readRequests ->
assertTrue(readRequests > 0));
+ getBookieReadEntrySchedulingDelayStats().values()
+ .forEach(readRequests -> assertTrue(readRequests > 0));
}
@Test
@@ -87,7 +89,7 @@ public class BookieStickyReadsTest extends
BookKeeperClusterTestCase {
writeAndReadEntries(conf, 3, 3, 3);
// All read requests should have been made to a single bookie
- Map<Integer, Long> stats = getBookieReadRequestStats();
+ Map<Integer, Long> stats = getBookieReadEntrySchedulingDelayStats();
boolean foundBookieWithRequests = false;
for (long readRequests : stats.values()) {
if (readRequests > 0) {
@@ -137,7 +139,7 @@ public class BookieStickyReadsTest extends
BookKeeperClusterTestCase {
// All read requests should have been made to a single bookie
int bookieWithRequests = -1;
for (int i = 0; i < NUM_BOOKIES; i++) {
- long requests =
getStatsProvider(i).getOpStatsLogger(READ_ENTRY_REQUEST_METRIC)
+ long requests =
getStatsProvider(i).getOpStatsLogger(READ_ENTRY_SCHEDULING_DELAY_METRIC)
.getSuccessCount();
log.info("Bookie {} --- requests: {}", i, requests);
@@ -162,18 +164,17 @@ public class BookieStickyReadsTest extends
BookKeeperClusterTestCase {
// At this point, we should have 1 bookie with 1 request (the initial
// request), and a second bookie with 10 requests. The 3rd bookie
should
// have no requests
- List<Long> requestCounts =
Lists.newArrayList(getBookieReadRequestStats().values());
+ List<Long> requestCounts =
Lists.newArrayList(getBookieReadEntrySchedulingDelayStats().values());
Collections.sort(requestCounts);
assertEquals(0, requestCounts.get(0).longValue());
assertEquals(1, requestCounts.get(1).longValue());
assertEquals(10, requestCounts.get(2).longValue());
}
-
- private Map<Integer, Long> getBookieReadRequestStats() throws Exception {
+ private Map<Integer, Long> getBookieReadEntrySchedulingDelayStats() throws
Exception {
Map<Integer, Long> stats = new TreeMap<>();
for (int i = 0; i < NUM_BOOKIES; i++) {
- stats.put(i,
getStatsProvider(i).getOpStatsLogger(READ_ENTRY_REQUEST_METRIC)
+ stats.put(i,
getStatsProvider(i).getOpStatsLogger(READ_ENTRY_SCHEDULING_DELAY_METRIC)
.getSuccessCount());
}