浏览代码

HDFS-16399. Reconfig cache report parameters for datanode (#3841)

litao 3 年之前
父节点
当前提交
e355646330

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java

@@ -113,7 +113,7 @@ public class DNConf {
   final long outliersReportIntervalMs;
   final long ibrInterval;
   final long initialBlockReportDelayMs;
-  final long cacheReportInterval;
+  volatile long cacheReportInterval;
   final long datanodeSlowIoWarningThresholdMs;
 
   final String minimumNameNodeVersion;
@@ -484,4 +484,14 @@ public class DNConf {
   public long getBlockReportInterval() {
     return blockReportInterval;
   }
+
+  void setCacheReportInterval(long intervalMs) {
+    Preconditions.checkArgument(intervalMs > 0,
+        "dfs.cachereport.intervalMsec should be larger than 0");
+    cacheReportInterval = intervalMs;
+  }
+
+  public long getCacheReportInterval() {
+    return cacheReportInterval;
+  }
 }

+ 23 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ALLOW_SAME_DISK_TIERING;
@@ -306,7 +308,8 @@ public class DataNode extends ReconfigurableBase
               DFS_DATANODE_DATA_DIR_KEY,
               DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY,
               DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
-              DFS_DATANODE_MAX_RECEIVER_THREADS_KEY));
+              DFS_DATANODE_MAX_RECEIVER_THREADS_KEY,
+              DFS_CACHEREPORT_INTERVAL_MSEC_KEY));
 
   public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog");
 
@@ -652,6 +655,8 @@ public class DataNode extends ReconfigurableBase
     }
     case DFS_DATANODE_MAX_RECEIVER_THREADS_KEY:
       return reconfDataXceiverParameters(property, newVal);
+    case DFS_CACHEREPORT_INTERVAL_MSEC_KEY:
+      return reconfCacheReportParameters(property, newVal);
     default:
       break;
     }
@@ -676,6 +681,23 @@ public class DataNode extends ReconfigurableBase
     }
   }
 
+  private String reconfCacheReportParameters(String property, String newVal)
+      throws ReconfigurationException {
+    String result;
+    try {
+      LOG.info("Reconfiguring {} to {}", property, newVal);
+      Preconditions.checkNotNull(dnConf, "DNConf has not been initialized.");
+      long reportInterval = (newVal == null ? DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT :
+          Long.parseLong(newVal));
+      result = Long.toString(reportInterval);
+      dnConf.setCacheReportInterval(reportInterval);
+      LOG.info("RECONFIGURE* changed {} to {}", property, newVal);
+      return result;
+    } catch (IllegalArgumentException e) {
+      throw new ReconfigurationException(property, newVal, getConf().get(property), e);
+    }
+  }
+
   /**
    * Get a list of the keys of the re-configurable properties in configuration.
    */

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT;
@@ -406,4 +408,45 @@ public class TestDataNodeReconfiguration {
           dn.getConf().get(DFS_DATANODE_MAX_RECEIVER_THREADS_KEY));
     }
   }
+
+  @Test
+  public void testCacheReportReconfiguration()
+      throws ReconfigurationException {
+    int cacheReportInterval = 300 * 1000;
+    for (int i = 0; i < NUM_DATA_NODE; i++) {
+      DataNode dn = cluster.getDataNodes().get(i);
+
+      // Try invalid values.
+      try {
+        dn.reconfigureProperty(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, "text");
+        fail("ReconfigurationException expected");
+      } catch (ReconfigurationException expected) {
+        assertTrue("expecting NumberFormatException",
+            expected.getCause() instanceof NumberFormatException);
+      }
+      try {
+        dn.reconfigureProperty(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, String.valueOf(-1));
+        fail("ReconfigurationException expected");
+      } catch (ReconfigurationException expected) {
+        assertTrue("expecting IllegalArgumentException",
+            expected.getCause() instanceof IllegalArgumentException);
+      }
+
+      // Change properties.
+      dn.reconfigureProperty(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
+          String.valueOf(cacheReportInterval));
+
+      // Verify change.
+      assertEquals(String.format("%s has wrong value", DFS_CACHEREPORT_INTERVAL_MSEC_KEY),
+          cacheReportInterval, dn.getDnConf().getCacheReportInterval());
+
+      // Revert to default.
+      dn.reconfigureProperty(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, null);
+      assertEquals(String.format("%s has wrong value", DFS_CACHEREPORT_INTERVAL_MSEC_KEY),
+          DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT, dn.getDnConf().getCacheReportInterval());
+
+      assertNull(String.format("expect %s is not configured", DFS_CACHEREPORT_INTERVAL_MSEC_KEY),
+          dn.getConf().get(DFS_CACHEREPORT_INTERVAL_MSEC_KEY));
+    }
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java

@@ -338,7 +338,7 @@ public class TestDFSAdmin {
     final List<String> outs = Lists.newArrayList();
     final List<String> errs = Lists.newArrayList();
     getReconfigurableProperties("datanode", address, outs, errs);
-    assertEquals(5, outs.size());
+    assertEquals(6, outs.size());
     assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, outs.get(1));
   }