HDFS-15892. Add metric for editPendingQ in FSEditLogAsync (#2770)

Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
This commit is contained in:
litao 2021-04-02 09:33:40 +08:00 committed by GitHub
parent 4f28738010
commit 4bd04126d6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 19 additions and 0 deletions

View File

@ -228,6 +228,7 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
| `EditLogTailIntervalNumOps` | Total number of intervals between edit log tailings by standby NameNode |
| `EditLogTailIntervalAvgTime` | Average time of intervals between edit log tailings by standby NameNode in milliseconds |
| `EditLogTailInterval`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time between edit log tailings by standby NameNode in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
| `PendingEditsCount` | Current number of pending edits |
FSNamesystem
------------

View File

@ -28,6 +28,8 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@ -53,6 +55,8 @@ class FSEditLogAsync extends FSEditLog implements Runnable {
// of the edit log buffer - ie. a sync will eventually be forced.
private final Deque<Edit> syncWaitQ = new ArrayDeque<Edit>();
private long lastFull = 0;
FSEditLogAsync(Configuration conf, NNStorage storage, List<URI> editsDirs) {
super(conf, storage, editsDirs);
// op instances cannot be shared due to queuing for background thread.
@ -194,6 +198,11 @@ private void enqueueEdit(Edit edit) {
if (!editPendingQ.offer(edit)) {
Preconditions.checkState(
isSyncThreadAlive(), "sync thread is not alive");
long now = Time.monotonicNow();
if (now - lastFull > 4000) {
lastFull = now;
LOG.info("Edit pending queue is full");
}
if (Thread.holdsLock(this)) {
// if queue is full, synchronized caller must immediately relinquish
// the monitor before re-offering to avoid deadlock with sync thread
@ -231,15 +240,18 @@ private Edit dequeueEdit() throws InterruptedException {
public void run() {
try {
while (true) {
NameNodeMetrics metrics = NameNode.getNameNodeMetrics();
boolean doSync;
Edit edit = dequeueEdit();
if (edit != null) {
// sync if requested by edit log.
doSync = edit.logEdit();
syncWaitQ.add(edit);
metrics.setPendingEditsCount(editPendingQ.size() + 1);
} else {
// sync when editq runs dry, but have edits pending a sync.
doSync = !syncWaitQ.isEmpty();
metrics.setPendingEditsCount(0);
}
if (doSync) {
// normally edit log exceptions cause the NN to terminate, but tests

View File

@ -87,6 +87,8 @@ public class NameNodeMetrics {
MutableGaugeInt blockOpsQueued;
@Metric("Number of blockReports and blockReceivedAndDeleted batch processed")
MutableCounterLong blockOpsBatched;
@Metric("Number of pending edits")
MutableGaugeInt pendingEditsCount;
@Metric("Number of file system operations")
public long totalFileOps(){
@ -343,6 +345,10 @@ public void addBlockOpsBatched(int count) {
blockOpsBatched.incr(count);
}
public void setPendingEditsCount(int size) {
pendingEditsCount.set(size);
}
public void addTransaction(long latency) {
transactions.add(latency);
}