HDFS-2510. Add HA-related metrics. Contributed by Aaron T. Myers.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1242410 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2012-02-09 17:39:50 +00:00
parent e918b91e23
commit bad0a2a4a1
4 changed files with 138 additions and 1 deletions

View File

@ -184,3 +184,5 @@ HDFS-2901. Improvements for SBN web UI - not show under-replicated/missing block
HDFS-2905. HA: Standby NN NPE when shared edits dir is deleted. (Bikas Saha via jitendra)
HDFS-2579. Starting delegation token manager during safemode fails. (todd)
HDFS-2510. Add HA-related metrics. (atm)

View File

@ -118,6 +118,7 @@ import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
@ -4241,16 +4242,34 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return blockManager.getExcessBlocksCount();
}
// HA-only metric
@Metric
public long getPostponedMisreplicatedBlocks() {
return blockManager.getPostponedMisreplicatedBlocksCount();
}
// HA-only metric
@Metric
public int getPendingDataNodeMessageCount() {
return blockManager.getPendingDataNodeMessageCount();
}
// HA-only metric
@Metric
public String getHAState() {
return haContext.getState().toString();
}
// HA-only metric
@Metric
public long getMillisSinceLastLoadedEdits() {
if (isInStandbyState() && editLogTailer != null) {
return now() - editLogTailer.getLastLoadTimestamp();
} else {
return 0;
}
}
@Metric
public int getBlockCapacity() {
return blockManager.getCapacity();

View File

@ -239,6 +239,13 @@ public class EditLogTailer {
}
}
/**
* @return timestamp (in msec) of when we last loaded a non-zero number of edits.
*/
public long getLastLoadTimestamp() {
return lastLoadTimestamp;
}
/**
* @return true if the configured log roll period has elapsed.
*/

View File

@ -0,0 +1,109 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.io.IOUtils;
import org.junit.Test;
/**
* Make sure HA-related metrics are updated and reported appropriately.
*/
public class TestHAMetrics {
private static final Log LOG = LogFactory.getLog(TestHAMetrics.class);
@Test
public void testHAMetrics() throws Exception {
Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, Integer.MAX_VALUE);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(1)
.build();
FileSystem fs = null;
try {
cluster.waitActive();
FSNamesystem nn0 = cluster.getNamesystem(0);
FSNamesystem nn1 = cluster.getNamesystem(1);
assertEquals(nn0.getHAState(), "standby");
assertTrue(0 < nn0.getMillisSinceLastLoadedEdits());
assertEquals(nn1.getHAState(), "standby");
assertTrue(0 < nn1.getMillisSinceLastLoadedEdits());
cluster.transitionToActive(0);
assertEquals("active", nn0.getHAState());
assertEquals(0, nn0.getMillisSinceLastLoadedEdits());
assertEquals("standby", nn1.getHAState());
assertTrue(0 < nn1.getMillisSinceLastLoadedEdits());
cluster.transitionToStandby(0);
cluster.transitionToActive(1);
assertEquals("standby", nn0.getHAState());
assertTrue(0 < nn0.getMillisSinceLastLoadedEdits());
assertEquals("active", nn1.getHAState());
assertEquals(0, nn1.getMillisSinceLastLoadedEdits());
Thread.sleep(2000); // make sure standby gets a little out-of-date
assertTrue(2000 <= nn0.getMillisSinceLastLoadedEdits());
assertEquals(0, nn0.getPendingDataNodeMessageCount());
assertEquals(0, nn1.getPendingDataNodeMessageCount());
fs = HATestUtil.configureFailoverFs(cluster, conf);
DFSTestUtil.createFile(fs, new Path("/foo"),
10, (short)1, 1L);
assertTrue(0 < nn0.getPendingDataNodeMessageCount());
assertEquals(0, nn1.getPendingDataNodeMessageCount());
long millisSinceLastLoadedEdits = nn0.getMillisSinceLastLoadedEdits();
HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(1),
cluster.getNameNode(0));
assertEquals(0, nn0.getPendingDataNodeMessageCount());
assertEquals(0, nn1.getPendingDataNodeMessageCount());
long newMillisSinceLastLoadedEdits = nn0.getMillisSinceLastLoadedEdits();
// Since we just waited for the standby to catch up, the time since we
// last loaded edits should be very low.
assertTrue("expected " + millisSinceLastLoadedEdits + " > " +
newMillisSinceLastLoadedEdits,
millisSinceLastLoadedEdits > newMillisSinceLastLoadedEdits);
} finally {
IOUtils.cleanup(LOG, fs);
cluster.shutdown();
}
}
}