HDFS-10469. Add number of active xceivers to datanode metrics. Contributed by Hanisha Koneru.
This commit is contained in:
parent
e98c0c7a1c
commit
dca298d79e
|
@ -1928,7 +1928,10 @@ public class DataNode extends ReconfigurableBase
|
|||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
|
||||
if (metrics != null) {
|
||||
metrics.setDataNodeActiveXceiversCount(0);
|
||||
}
|
||||
|
||||
// IPC server needs to be shutdown late in the process, otherwise
|
||||
// shutdown command response won't get sent.
|
||||
if (ipcServer != null) {
|
||||
|
|
|
@ -232,11 +232,13 @@ class DataXceiverServer implements Runnable {
|
|||
}
|
||||
peers.put(peer, t);
|
||||
peersXceiver.put(peer, xceiver);
|
||||
datanode.metrics.incrDataNodeActiveXceiversCount();
|
||||
}
|
||||
|
||||
synchronized void closePeer(Peer peer) {
|
||||
peers.remove(peer);
|
||||
peersXceiver.remove(peer);
|
||||
datanode.metrics.decrDataNodeActiveXceiversCount();
|
||||
IOUtils.cleanup(null, peer);
|
||||
}
|
||||
|
||||
|
@ -282,6 +284,7 @@ class DataXceiverServer implements Runnable {
|
|||
}
|
||||
peers.clear();
|
||||
peersXceiver.clear();
|
||||
datanode.metrics.setDataNodeActiveXceiversCount(0);
|
||||
}
|
||||
|
||||
// Return the number of peers.
|
||||
|
@ -303,6 +306,7 @@ class DataXceiverServer implements Runnable {
|
|||
synchronized void releasePeer(Peer peer) {
|
||||
peers.remove(peer);
|
||||
peersXceiver.remove(peer);
|
||||
datanode.metrics.decrDataNodeActiveXceiversCount();
|
||||
}
|
||||
|
||||
public void updateBalancerMaxConcurrentMovers(int movers) {
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
|
|||
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
||||
import org.apache.hadoop.metrics2.lib.MutableQuantiles;
|
||||
import org.apache.hadoop.metrics2.lib.MutableRate;
|
||||
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
|
||||
import org.apache.hadoop.metrics2.source.JvmMetrics;
|
||||
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
@ -101,6 +102,9 @@ public class DataNodeMetrics {
|
|||
@Metric("Count of network errors on the datanode")
|
||||
MutableCounterLong datanodeNetworkErrors;
|
||||
|
||||
@Metric("Count of active dataNode xceivers")
|
||||
private MutableGaugeInt dataNodeActiveXceiversCount;
|
||||
|
||||
@Metric MutableRate readBlockOp;
|
||||
@Metric MutableRate writeBlockOp;
|
||||
@Metric MutableRate blockChecksumOp;
|
||||
|
@ -429,4 +433,16 @@ public class DataNodeMetrics {
|
|||
ecFailedReconstructionTasks.incr();
|
||||
}
|
||||
|
||||
public void incrDataNodeActiveXceiversCount() {
|
||||
dataNodeActiveXceiversCount.incr();
|
||||
}
|
||||
|
||||
public void decrDataNodeActiveXceiversCount() {
|
||||
dataNodeActiveXceiversCount.decr();
|
||||
}
|
||||
|
||||
public void setDataNodeActiveXceiversCount(int value) {
|
||||
this.dataNodeActiveXceiversCount.set(value);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.MetricsAsserts;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -331,4 +332,34 @@ public class TestDataNodeMetrics {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDatanodeActiveXceiversCount() throws Exception {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
||||
try {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
List<DataNode> datanodes = cluster.getDataNodes();
|
||||
assertEquals(datanodes.size(), 1);
|
||||
DataNode datanode = datanodes.get(0);
|
||||
|
||||
MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name());
|
||||
long dataNodeActiveXceiversCount = MetricsAsserts.getIntGauge(
|
||||
"DataNodeActiveXceiversCount", rb);
|
||||
assertEquals(dataNodeActiveXceiversCount, 0);
|
||||
|
||||
Path path = new Path("/counter.txt");
|
||||
DFSTestUtil.createFile(fs, path, 204800000, (short) 3, Time
|
||||
.monotonicNow());
|
||||
|
||||
MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name());
|
||||
dataNodeActiveXceiversCount = MetricsAsserts.getIntGauge(
|
||||
"DataNodeActiveXceiversCount", rbNew);
|
||||
assertTrue(dataNodeActiveXceiversCount >= 0);
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue