HDFS-4185. Add a metric for number of active leases (Rakesh R via raviprak)

(cherry picked from commit cdfae446ad)
This commit is contained in:
Ravi Prakash 2015-05-18 12:37:21 -07:00
parent 7bd4db968d
commit e5b805d361
4 changed files with 83 additions and 4 deletions

View File

@ -233,6 +233,8 @@ Release 2.8.0 - UNRELEASED
HDFS-8345. Storage policy APIs must be exposed via the FileSystem
interface. (Arpit Agarwal)
HDFS-4185. Add a metric for number of active leases (Rakesh R via raviprak)
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -5340,6 +5340,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return blockManager.getTotalBlocks();
}
/**
* Get the number of files under construction in the system.
*/
@Metric({ "NumFilesUnderConstruction",
"Number of files under construction" })
public long getNumFilesUnderConstruction() {
return leaseManager.countPath();
}
/**
* Get the total number of active clients holding lease in the system.
*/
@Metric({ "NumActiveClients", "Number of active clients holding lease" })
public long getNumActiveClients() {
return leaseManager.countLease();
}
/**
* Get the total number of COMPLETE blocks in the system.
* For safe mode only complete blocks are counted.

View File

@ -130,6 +130,15 @@ public class LeaseManager {
@VisibleForTesting
public synchronized int countLease() {return sortedLeases.size();}
/** @return the number of paths contained in all leases */
synchronized int countPath() {
int count = 0;
for (Lease lease : sortedLeases) {
count += lease.getFiles().size();
}
return count;
}
/**
* Adds (or re-adds) the lease for the specified file.
*/

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hdfs.server.namenode.metrics;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges;
@ -29,12 +28,12 @@ import java.io.DataInputStream;
import java.io.IOException;
import java.util.Random;
import com.google.common.collect.ImmutableList;
import com.google.common.io.Files;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.Path;
@ -55,12 +54,10 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.MetricsSource;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.test.MetricsAsserts;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level;
import org.junit.After;
import org.junit.Before;
@ -568,4 +565,58 @@ public class TestNameNodeMetrics {
assertTrue(MetricsAsserts.getLongCounter("TransactionsNumOps", rbNew) >
startWriteCounter);
}
/**
* Test metrics indicating the number of active clients and the files under
* construction
*/
@Test(timeout = 60000)
public void testNumActiveClientsAndFilesUnderConstructionMetrics()
throws Exception {
final Path file1 = getTestPath("testFileAdd1");
createFile(file1, 100, (short) 3);
assertGauge("NumActiveClients", 0L, getMetrics(NS_METRICS));
assertGauge("NumFilesUnderConstruction", 0L, getMetrics(NS_METRICS));
Path file2 = new Path("/testFileAdd2");
FSDataOutputStream output2 = fs.create(file2);
output2.writeBytes("Some test data");
assertGauge("NumActiveClients", 1L, getMetrics(NS_METRICS));
assertGauge("NumFilesUnderConstruction", 1L, getMetrics(NS_METRICS));
Path file3 = new Path("/testFileAdd3");
FSDataOutputStream output3 = fs.create(file3);
output3.writeBytes("Some test data");
assertGauge("NumActiveClients", 1L, getMetrics(NS_METRICS));
assertGauge("NumFilesUnderConstruction", 2L, getMetrics(NS_METRICS));
// create another DistributedFileSystem client
DistributedFileSystem fs1 = (DistributedFileSystem) cluster
.getNewFileSystemInstance(0);
try {
Path file4 = new Path("/testFileAdd4");
FSDataOutputStream output4 = fs1.create(file4);
output4.writeBytes("Some test data");
assertGauge("NumActiveClients", 2L, getMetrics(NS_METRICS));
assertGauge("NumFilesUnderConstruction", 3L, getMetrics(NS_METRICS));
Path file5 = new Path("/testFileAdd35");
FSDataOutputStream output5 = fs1.create(file5);
output5.writeBytes("Some test data");
assertGauge("NumActiveClients", 2L, getMetrics(NS_METRICS));
assertGauge("NumFilesUnderConstruction", 4L, getMetrics(NS_METRICS));
output2.close();
output3.close();
assertGauge("NumActiveClients", 1L, getMetrics(NS_METRICS));
assertGauge("NumFilesUnderConstruction", 2L, getMetrics(NS_METRICS));
output4.close();
output5.close();
assertGauge("NumActiveClients", 0L, getMetrics(NS_METRICS));
assertGauge("NumFilesUnderConstruction", 0L, getMetrics(NS_METRICS));
} finally {
fs1.close();
}
}
}