|
@@ -23,13 +23,18 @@ import java.io.BufferedOutputStream;
|
|
|
import java.io.ByteArrayInputStream;
|
|
|
import java.io.DataInputStream;
|
|
|
import java.io.IOException;
|
|
|
+import java.lang.System;
|
|
|
import java.net.Socket;
|
|
|
import java.nio.ByteBuffer;
|
|
|
+import java.util.Date;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.Objects;
|
|
|
import java.util.Queue;
|
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+import java.util.LinkedHashMap;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
|
|
|
|
@@ -99,6 +104,24 @@ public class LearnerHandler extends ZooKeeperThread {
|
|
|
*/
|
|
|
final LinkedBlockingQueue<QuorumPacket> queuedPackets =
|
|
|
new LinkedBlockingQueue<QuorumPacket>();
|
|
|
+ private final AtomicLong queuedPacketsSize = new AtomicLong();
|
|
|
+
|
|
|
+ protected final AtomicLong packetsReceived = new AtomicLong();
|
|
|
+ protected final AtomicLong packetsSent = new AtomicLong();
|
|
|
+
|
|
|
+ protected final AtomicLong requestsReceived = new AtomicLong();
|
|
|
+
|
|
|
+ protected volatile long lastZxid = -1;
|
|
|
+
|
|
|
+ public synchronized long getLastZxid() {
|
|
|
+ return lastZxid;
|
|
|
+ }
|
|
|
+
|
|
|
+ protected final Date established = new Date();
|
|
|
+
|
|
|
+ public Date getEstablished() {
|
|
|
+ return (Date)established.clone();
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Marker packets would be added to quorum packet queue after every
|
|
@@ -297,6 +320,7 @@ public class LearnerHandler extends ZooKeeperThread {
|
|
|
continue;
|
|
|
}
|
|
|
|
|
|
+ queuedPacketsSize.addAndGet(-packetSize(p));
|
|
|
if (p == proposalOfDeath) {
|
|
|
// Packet of death!
|
|
|
break;
|
|
@@ -310,7 +334,13 @@ public class LearnerHandler extends ZooKeeperThread {
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
ZooTrace.logQuorumPacket(LOG, traceMask, 'o', p);
|
|
|
}
|
|
|
+
|
|
|
+ // Log the zxid of the last request, if it is a valid zxid.
|
|
|
+ if (p.getZxid() > 0) {
|
|
|
+ lastZxid = p.getZxid();
|
|
|
+ }
|
|
|
oa.writeRecord(p, "packet");
|
|
|
+ packetsSent.incrementAndGet();
|
|
|
} catch (IOException e) {
|
|
|
if (!sock.isClosed()) {
|
|
|
LOG.warn("Unexpected exception at " + this, e);
|
|
@@ -602,6 +632,7 @@ public class LearnerHandler extends ZooKeeperThread {
|
|
|
}
|
|
|
tickOfNextAckDeadline = learnerMaster.getTickOfNextAckDeadline();
|
|
|
|
|
|
+ packetsReceived.incrementAndGet();
|
|
|
|
|
|
ByteBuffer bb;
|
|
|
long sessionId;
|
|
@@ -647,6 +678,7 @@ public class LearnerHandler extends ZooKeeperThread {
|
|
|
}
|
|
|
si.setOwner(this);
|
|
|
learnerMaster.submitLearnerRequest(si);
|
|
|
+ requestsReceived.incrementAndGet();
|
|
|
break;
|
|
|
default:
|
|
|
LOG.warn("unexpected quorum packet, type: {}", packetToString(qp));
|
|
@@ -1025,6 +1057,7 @@ public class LearnerHandler extends ZooKeeperThread {
|
|
|
packetCounter.getAndIncrement() % markerPacketInterval == 0) {
|
|
|
queuedPackets.add(new MarkerQuorumPacket(System.nanoTime()));
|
|
|
}
|
|
|
+ queuedPacketsSize.addAndGet(packetSize(p));
|
|
|
}
|
|
|
|
|
|
static long packetSize(QuorumPacket p) {
|
|
@@ -1041,6 +1074,29 @@ public class LearnerHandler extends ZooKeeperThread {
|
|
|
return isAlive() && learnerMaster.getCurrentTick() <= tickOfNextAckDeadline;
|
|
|
}
|
|
|
|
|
|
+ public synchronized Map<String, Object> getLearnerHandlerInfo() {
|
|
|
+ Map<String, Object> info = new LinkedHashMap<>(9);
|
|
|
+ info.put("remote_socket_address", getRemoteAddress());
|
|
|
+ info.put("sid", getSid());
|
|
|
+ info.put("established", getEstablished());
|
|
|
+ info.put("queued_packets", queuedPackets.size());
|
|
|
+ info.put("queued_packets_size", queuedPacketsSize.get());
|
|
|
+ info.put("packets_received", packetsReceived.longValue());
|
|
|
+ info.put("packets_sent", packetsSent.longValue());
|
|
|
+ info.put("requests", requestsReceived.longValue());
|
|
|
+ info.put("last_zxid", getLastZxid());
|
|
|
+
|
|
|
+ return info;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized void resetObserverConnectionStats() {
|
|
|
+ packetsReceived.set(0);
|
|
|
+ packetsSent.set(0);
|
|
|
+ requestsReceived.set(0);
|
|
|
+
|
|
|
+ lastZxid = -1;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* For testing, return packet queue
|
|
|
* @return
|