This is an automated email from the ASF dual-hosted git repository.
sijie 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 20db5fe PendingReadOp: improve logging for speculative read targets
20db5fe is described below
commit 20db5fe96571a44429cdf0fad0dfd32240c55c51
Author: Samuel Just <[email protected]>
AuthorDate: Thu Jan 18 11:09:05 2018 -0800
PendingReadOp: improve logging for speculative read targets
And add more log messages around speculative read
to understand what bookies were tried, before attempting
speculative read.
(bug W-3315760)
Signed-off-by: Venkateswararao Jujjuri (JV) <vjujjurisalesforce.com>
[Fixed merge conflicts and checkstyle issues]
Signed-off-by: Samuel Just <sjustsalesforce.com>
Author: Samuel Just <[email protected]>
Reviewers: Enrico Olivelli <[email protected]>, Sijie Guo
<[email protected]>
This closes #993 from athanatos/forupstream/stats1/specreadlogs
---
.../org/apache/bookkeeper/client/PendingReadOp.java | 17 +++++++++++------
1 file changed, 11 insertions(+), 6 deletions(-)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
index 5f1cecd..2e59e35 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
@@ -62,8 +62,9 @@ class PendingReadOp implements ReadEntryCallback,
SafeRunnable {
private ScheduledFuture<?> speculativeTask = null;
protected final List<LedgerEntryRequest> seq;
private final CompletableFuture<LedgerEntries> future;
- Set<BookieSocketAddress> heardFromHosts;
- BitSet heardFromHostsBitSet;
+ private final Set<BookieSocketAddress> heardFromHosts;
+ private final BitSet heardFromHostsBitSet;
+ private final Set<BookieSocketAddress> sentToHosts = new
HashSet<BookieSocketAddress>();
LedgerHandle lh;
long numPendingEntries;
long startEntryId;
@@ -257,8 +258,9 @@ class PendingReadOp implements ReadEntryCallback,
SafeRunnable {
public Boolean call() throws Exception {
if (!isComplete() && null !=
maybeSendSpeculativeRead(heardFromHostsBitSet)) {
if (LOG.isDebugEnabled()) {
- LOG.debug("Send speculative read for {}. Hosts
heard are {}, ensemble is {}.",
- this, heardFromHostsBitSet, ensemble);
+ LOG.debug("Send speculative read for {}. Hosts
sent are {}, "
+ + " Hosts heard are {}, ensemble
is {}.",
+ this, sentToHosts, heardFromHostsBitSet,
ensemble);
}
return true;
}
@@ -399,6 +401,7 @@ class PendingReadOp implements ReadEntryCallback,
SafeRunnable {
try {
BookieSocketAddress to = ensemble.get(bookieIndex);
sendReadTo(bookieIndex, to, this);
+ sentToHosts.add(to);
sentReplicas.set(replica);
return to;
} catch (InterruptedException ie) {
@@ -615,8 +618,10 @@ class PendingReadOp implements ReadEntryCallback,
SafeRunnable {
break;
}
}
- LOG.error("Read of ledger entry failed: L{} E{}-E{}, Heard from {}
: bitset = {}. First unread entry is {}",
- lh.getId(), startEntryId, endEntryId, heardFromHosts,
heardFromHostsBitSet,
+ LOG.error(
+ "Read of ledger entry failed: L{} E{}-E{}, Sent to {}, "
+ + "Heard from {} : bitset = {}. First unread entry
is {}",
+ lh.getId(), startEntryId, endEntryId, sentToHosts,
heardFromHosts, heardFromHostsBitSet,
firstUnread);
readOpLogger.registerFailedEvent(latencyNanos,
TimeUnit.NANOSECONDS);
// release the entries
--
To stop receiving notification emails like this one, please contact
['"[email protected]" <[email protected]>'].