|
@@ -19,11 +19,14 @@
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.junit.Test;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.After;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY;
|
|
|
import static org.junit.Assert.*;
|
|
|
|
|
@@ -40,7 +43,9 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.SlowPeerTracker;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
|
|
@@ -513,6 +518,55 @@ public class TestNameNodeReconfigure {
|
|
|
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testSlowPeerMaxNodesToReportReconf() throws Exception {
|
|
|
+ final NameNode nameNode = cluster.getNameNode();
|
|
|
+ final DatanodeManager datanodeManager = nameNode.namesystem.getBlockManager()
|
|
|
+ .getDatanodeManager();
|
|
|
+ nameNode.reconfigurePropertyImpl(DFS_DATANODE_PEER_STATS_ENABLED_KEY, "true");
|
|
|
+ assertTrue("SlowNode tracker is still disabled. Reconfiguration could not be successful",
|
|
|
+ datanodeManager.getSlowPeerTracker().isSlowPeerTrackerEnabled());
|
|
|
+
|
|
|
+ SlowPeerTracker tracker = datanodeManager.getSlowPeerTracker();
|
|
|
+
|
|
|
+ OutlierMetrics outlierMetrics1 = new OutlierMetrics(0.0, 0.0, 0.0, 1.1);
|
|
|
+ tracker.addReport("node1", "node70", outlierMetrics1);
|
|
|
+ OutlierMetrics outlierMetrics2 = new OutlierMetrics(0.0, 0.0, 0.0, 1.23);
|
|
|
+ tracker.addReport("node2", "node71", outlierMetrics2);
|
|
|
+ OutlierMetrics outlierMetrics3 = new OutlierMetrics(0.0, 0.0, 0.0, 2.13);
|
|
|
+ tracker.addReport("node3", "node72", outlierMetrics3);
|
|
|
+ OutlierMetrics outlierMetrics4 = new OutlierMetrics(0.0, 0.0, 0.0, 1.244);
|
|
|
+ tracker.addReport("node4", "node73", outlierMetrics4);
|
|
|
+ OutlierMetrics outlierMetrics5 = new OutlierMetrics(0.0, 0.0, 0.0, 0.2);
|
|
|
+ tracker.addReport("node5", "node74", outlierMetrics4);
|
|
|
+ OutlierMetrics outlierMetrics6 = new OutlierMetrics(0.0, 0.0, 0.0, 1.244);
|
|
|
+ tracker.addReport("node6", "node75", outlierMetrics4);
|
|
|
+
|
|
|
+ String jsonReport = tracker.getJson();
|
|
|
+ LOG.info("Retrieved slow peer json report: {}", jsonReport);
|
|
|
+
|
|
|
+ List<Boolean> containReport = validatePeerReport(jsonReport);
|
|
|
+ assertEquals(1, containReport.stream().filter(reportVal -> !reportVal).count());
|
|
|
+
|
|
|
+ nameNode.reconfigurePropertyImpl(DFS_DATANODE_MAX_NODES_TO_REPORT_KEY, "2");
|
|
|
+ jsonReport = tracker.getJson();
|
|
|
+ LOG.info("Retrieved slow peer json report: {}", jsonReport);
|
|
|
+
|
|
|
+ containReport = validatePeerReport(jsonReport);
|
|
|
+ assertEquals(4, containReport.stream().filter(reportVal -> !reportVal).count());
|
|
|
+ }
|
|
|
+
|
|
|
+ private List<Boolean> validatePeerReport(String jsonReport) {
|
|
|
+ List<Boolean> containReport = new ArrayList<>();
|
|
|
+ containReport.add(jsonReport.contains("node1"));
|
|
|
+ containReport.add(jsonReport.contains("node2"));
|
|
|
+ containReport.add(jsonReport.contains("node3"));
|
|
|
+ containReport.add(jsonReport.contains("node4"));
|
|
|
+ containReport.add(jsonReport.contains("node5"));
|
|
|
+ containReport.add(jsonReport.contains("node6"));
|
|
|
+ return containReport;
|
|
|
+ }
|
|
|
+
|
|
|
@After
|
|
|
public void shutDown() throws IOException {
|
|
|
if (cluster != null) {
|