Bladeren bron

HDFS-9847. HDFS configuration should accept time units. Contributed by Yiqun Lin

Chris Douglas 8 jaren geleden
bovenliggende
commit
d37dc5d1b8
17 gewijzigde bestanden met toevoegingen van 187 en 66 verwijderingen
  1. 34 13
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  2. 46 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  3. 4 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
  4. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  5. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  6. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  7. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  8. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
  9. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
  10. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  11. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  12. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  13. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
  14. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  15. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
  16. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  17. 35 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

+ 34 - 13
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -630,10 +630,15 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     ArrayList<String > names = new ArrayList<String>();
 	if (isDeprecated(name)) {
       DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name);
-      warnOnceIfDeprecated(deprecations, name);
-      for (String newKey : keyInfo.newKeys) {
-        if(newKey != null) {
-          names.add(newKey);
+      if (keyInfo != null) {
+        if (!keyInfo.getAndSetAccessed()) {
+          logDeprecation(keyInfo.getWarningMessage(name));
+        }
+
+        for (String newKey : keyInfo.newKeys) {
+          if (newKey != null) {
+            names.add(newKey);
+          }
         }
       }
     }
@@ -1232,11 +1237,9 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
-  private void warnOnceIfDeprecated(DeprecationContext deprecations, String name) {
-    DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name);
-    if (keyInfo != null && !keyInfo.getAndSetAccessed()) {
-      LOG_DEPRECATION.info(keyInfo.getWarningMessage(name));
-    }
+  @VisibleForTesting
+  void logDeprecation(String message) {
+    LOG_DEPRECATION.info(message);
   }
 
   /**
@@ -1625,20 +1628,38 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     String vStr = get(name);
     if (null == vStr) {
       return defaultValue;
+    } else {
+      return getTimeDurationHelper(name, vStr, unit);
+    }
+  }
+
+  public long getTimeDuration(String name, String defaultValue, TimeUnit unit) {
+    String vStr = get(name);
+    if (null == vStr) {
+      return getTimeDurationHelper(name, defaultValue, unit);
+    } else {
+      return getTimeDurationHelper(name, vStr, unit);
     }
-    vStr = vStr.trim();
-    return getTimeDurationHelper(name, vStr, unit);
   }
 
   private long getTimeDurationHelper(String name, String vStr, TimeUnit unit) {
+    vStr = vStr.trim();
+    vStr = StringUtils.toLowerCase(vStr);
     ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr);
     if (null == vUnit) {
-      LOG.warn("No unit for " + name + "(" + vStr + ") assuming " + unit);
+      logDeprecation("No unit for " + name + "(" + vStr + ") assuming " + unit);
       vUnit = ParsedTimeDuration.unitFor(unit);
     } else {
       vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix()));
     }
-    return unit.convert(Long.parseLong(vStr), vUnit.unit());
+
+    long raw = Long.parseLong(vStr);
+    long converted = unit.convert(raw, vUnit.unit());
+    if (vUnit.unit().convert(converted, unit) < raw) {
+      logDeprecation("Possible loss of precision converting " + vStr
+          + vUnit.suffix() + " to " + unit + " for " + name);
+    }
+    return converted;
   }
 
   public long[] getTimeDurations(String name, TimeUnit unit) {

+ 46 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -880,6 +880,15 @@ public class TestConfiguration extends TestCase {
     assertEquals(30L, conf.getTimeDuration("test.time.X", 30, SECONDS));
     conf.set("test.time.X", "30");
     assertEquals(30L, conf.getTimeDuration("test.time.X", 40, SECONDS));
+    assertEquals(10L, conf.getTimeDuration("test.time.c", "10", SECONDS));
+    assertEquals(30L, conf.getTimeDuration("test.time.c", "30s", SECONDS));
+    assertEquals(120L, conf.getTimeDuration("test.time.c", "2m", SECONDS));
+    conf.set("test.time.c", "30");
+    assertEquals(30L, conf.getTimeDuration("test.time.c", "40s", SECONDS));
+
+    // check suffix insensitive
+    conf.set("test.time.d", "30S");
+    assertEquals(30L, conf.getTimeDuration("test.time.d", 40, SECONDS));
 
     for (Configuration.ParsedTimeDuration ptd :
          Configuration.ParsedTimeDuration.values()) {
@@ -889,6 +898,43 @@ public class TestConfiguration extends TestCase {
     }
   }
 
+  public void testTimeDurationWarning() {
+    // check warn for possible loss of precision
+    final String warnFormat = "Possible loss of precision converting %s" +
+            " to %s for test.time.warn";
+    final ArrayList<String> warnchk = new ArrayList<>();
+    Configuration wconf = new Configuration(false) {
+      @Override
+      void logDeprecation(String message) {
+        warnchk.add(message);
+      }
+    };
+    String[] convDAYS = new String[]{"23h", "30m", "40s", "10us", "40000ms"};
+    for (String s : convDAYS) {
+      wconf.set("test.time.warn", s);
+      assertEquals(0, wconf.getTimeDuration("test.time.warn", 1, DAYS));
+    }
+    for (int i = 0; i < convDAYS.length; ++i) {
+      String wchk = String.format(warnFormat, convDAYS[i], "DAYS");
+      assertEquals(wchk, warnchk.get(i));
+    }
+
+    warnchk.clear();
+    wconf.setTimeDuration("test.time.warn", 1441, MINUTES);
+    assertEquals(1, wconf.getTimeDuration("test.time.warn", 0, DAYS));
+    assertEquals(24, wconf.getTimeDuration("test.time.warn", 0, HOURS));
+    String dchk = String.format(warnFormat, "1441m", "DAYS");
+    assertEquals(dchk, warnchk.get(0));
+    String hchk = String.format(warnFormat, "1441m", "HOURS");
+    assertEquals(hchk, warnchk.get(1));
+    assertEquals(1441, wconf.getTimeDuration("test.time.warn", 0, MINUTES));
+    // no warning
+    assertEquals(2, warnchk.size());
+    assertEquals(86460, wconf.getTimeDuration("test.time.warn", 0, SECONDS));
+    // no warning
+    assertEquals(2, warnchk.size());
+  }
+
   public void testPattern() throws IOException {
     out = new BufferedWriter(new FileWriter(CONFIG));
     startConfig();

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java

@@ -85,6 +85,7 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 /**
  * DFSClient configuration.
@@ -233,9 +234,10 @@ public class DfsClientConf {
     connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
         DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
 
-    datanodeRestartTimeout = conf.getLong(
+    datanodeRestartTimeout = conf.getTimeDuration(
         DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
-        DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT) * 1000;
+        DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT,
+        TimeUnit.SECONDS) * 1000;
     slowIoWarningThresholdMs = conf.getLong(
         DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
         DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -33,6 +33,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -662,10 +663,13 @@ public class Balancer {
   static int run(Collection<URI> namenodes, final BalancerParameters p,
       Configuration conf) throws IOException, InterruptedException {
     final long sleeptime =
-        conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
-            DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 +
-        conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
-            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
+        conf.getTimeDuration(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+            DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT,
+            TimeUnit.SECONDS) * 2000 +
+        conf.getTimeDuration(
+            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
+            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT,
+            TimeUnit.SECONDS) * 1000;
     LOG.info("namenodes  = " + namenodes);
     LOG.info("parameters = " + p);
     LOG.info("included nodes = " + p.getIncludedNodes());

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -42,6 +42,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import javax.management.ObjectName;
@@ -405,8 +406,9 @@ public class BlockManager implements BlockStatsMXBean {
     this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
 
     this.replicationRecheckInterval = 
-      conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 
-                  DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000L;
+      conf.getTimeDuration(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
+          DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT,
+          TimeUnit.SECONDS) * 1000L;
 
     this.storageInfoDefragmentInterval =
       conf.getLong(

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -89,9 +90,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     this.stats = stats;
     this.clusterMap = clusterMap;
     this.host2datanodeMap = host2datanodeMap;
-    this.heartbeatInterval = conf.getLong(
+    this.heartbeatInterval = conf.getTimeDuration(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
-        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 1000;
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000;
     this.tolerateHeartbeatMultiplier = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_KEY,
         DFSConfigKeys.DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_DEFAULT);

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -58,6 +58,7 @@ import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Manage datanodes, include decommission and other activities.
@@ -234,9 +235,9 @@ public class DatanodeManager {
       dnsToSwitchMapping.resolve(locations);
     }
 
-    heartbeatIntervalSeconds = conf.getLong(
+    heartbeatIntervalSeconds = conf.getTimeDuration(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
-        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT);
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
     heartbeatRecheckInterval = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 
         DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT); // 5 minutes
@@ -289,9 +290,9 @@ public class DatanodeManager {
         " = '" + staleInterval + "' is invalid. " +
         "It should be a positive non-zero value.");
     
-    final long heartbeatIntervalSeconds = conf.getLong(
+    final long heartbeatIntervalSeconds = conf.getTimeDuration(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
-        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT);
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
     // The stale interval value cannot be smaller than 
     // 3 times of heartbeat interval 
     final long minStaleInterval = conf.getInt(

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java

@@ -134,9 +134,10 @@ public class DecommissionManager {
    * @param conf
    */
   void activate(Configuration conf) {
-    final int intervalSecs =
-        conf.getInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY,
-            DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT);
+    final int intervalSecs = (int) conf.getTimeDuration(
+        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY,
+        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT,
+        TimeUnit.SECONDS);
     checkArgument(intervalSecs >= 0, "Cannot set a negative " +
         "value for " + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY);
 

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java

@@ -64,6 +64,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
 import org.apache.hadoop.security.SaslPropertiesResolver;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * Simple class encapsulating all of the configuration that the DataNode
  * loads at startup time.
@@ -184,9 +186,9 @@ public class DNConf {
         DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY,
         DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
 
-    long initBRDelay = conf.getLong(
+    long initBRDelay = conf.getTimeDuration(
         DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
-        DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT) * 1000L;
+        DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT, TimeUnit.SECONDS) * 1000L;
     if (initBRDelay >= blockReportInterval) {
       initBRDelay = 0;
       DataNode.LOG.info("dfs.blockreport.initialDelay is "
@@ -195,12 +197,12 @@ public class DNConf {
     }
     initialBlockReportDelayMs = initBRDelay;
     
-    heartBeatInterval = conf.getLong(DFS_HEARTBEAT_INTERVAL_KEY,
-        DFS_HEARTBEAT_INTERVAL_DEFAULT) * 1000L;
+    heartBeatInterval = conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
+        DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000L;
     long confLifelineIntervalMs =
         conf.getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY,
-        3 * conf.getLong(DFS_HEARTBEAT_INTERVAL_KEY,
-            DFS_HEARTBEAT_INTERVAL_DEFAULT)) * 1000L;
+        3 * conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
+        DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS)) * 1000L;
     if (confLifelineIntervalMs <= heartBeatInterval) {
       confLifelineIntervalMs = 3 * heartBeatInterval;
       DataNode.LOG.warn(
@@ -245,9 +247,9 @@ public class DNConf {
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST,
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST_DEFAULT);
 
-    this.bpReadyTimeout = conf.getLong(
+    this.bpReadyTimeout = conf.getTimeDuration(
         DFS_DATANODE_BP_READY_TIMEOUT_KEY,
-        DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT);
+        DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT, TimeUnit.SECONDS);
 
     this.volFailuresTolerated =
         conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,

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

@@ -1024,8 +1024,8 @@ public class DataNode extends ReconfigurableBase
       return;
     }
     String reason = null;
-    if (conf.getInt(DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 
-                    DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT) < 0) {
+    if (conf.getTimeDuration(DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
+        DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT, TimeUnit.SECONDS) < 0) {
       reason = "verification is turned off by configuration";
     } else if ("SimulatedFSDataset".equals(data.getClass().getSimpleName())) {
       reason = "verifcation is not supported by SimulatedFSDataset";

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -414,8 +414,10 @@ public class DirectoryScanner implements Runnable {
   DirectoryScanner(DataNode datanode, FsDatasetSpi<?> dataset, Configuration conf) {
     this.datanode = datanode;
     this.dataset = dataset;
-    int interval = conf.getInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
-        DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT);
+    int interval = (int) conf.getTimeDuration(
+        DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY,
+        DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT,
+        TimeUnit.SECONDS);
     scanPeriodMsecs = interval * MILLIS_PER_SECOND; //msec
 
     int throttle =

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java

@@ -64,6 +64,7 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.text.DateFormat;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 @InterfaceAudience.Private
@@ -602,10 +603,13 @@ public class Mover {
   static int run(Map<URI, List<Path>> namenodes, Configuration conf)
       throws IOException, InterruptedException {
     final long sleeptime =
-        conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
-            DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 +
-        conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
-            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
+        conf.getTimeDuration(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+            DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT,
+            TimeUnit.SECONDS) * 2000 +
+        conf.getTimeDuration(
+            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
+            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT,
+            TimeUnit.SECONDS) * 1000;
     AtomicInteger retryCount = new AtomicInteger(0);
     LOG.info("namenodes = " + namenodes);
 

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.conf.Configuration;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.concurrent.TimeUnit;
+
 @InterfaceAudience.Private
 public class CheckpointConf {
   private static final Log LOG = LogFactory.getLog(CheckpointConf.class);
@@ -52,12 +54,12 @@ public class CheckpointConf {
   private double quietMultiplier;
 
   public CheckpointConf(Configuration conf) {
-    checkpointCheckPeriod = conf.getLong(
+    checkpointCheckPeriod = conf.getTimeDuration(
         DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY,
-        DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT);
+        DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT, TimeUnit.SECONDS);
         
-    checkpointPeriod = conf.getLong(DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 
-                                    DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
+    checkpointPeriod = conf.getTimeDuration(DFS_NAMENODE_CHECKPOINT_PERIOD_KEY,
+        DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT, TimeUnit.SECONDS);
     checkpointTxnCount = conf.getLong(DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 
                                   DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
     maxRetriesOnMergeError = conf.getInt(DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY,

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -34,6 +34,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -798,9 +799,9 @@ public class FSImage implements Closeable {
    */
   private boolean needsResaveBasedOnStaleCheckpoint(
       File imageFile, long numEditsLoaded) {
-    final long checkpointPeriod = conf.getLong(
-        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 
-        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
+    final long checkpointPeriod = conf.getTimeDuration(
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY,
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT, TimeUnit.SECONDS);
     final long checkpointTxnCount = conf.getLong(
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java

@@ -146,8 +146,9 @@ public class EditLogTailer {
     
     lastLoadTimeMs = monotonicNow();
 
-    logRollPeriodMs = conf.getInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
-        DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT) * 1000;
+    logRollPeriodMs = conf.getTimeDuration(
+        DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
+        DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT, TimeUnit.SECONDS) * 1000;
     List<RemoteNameNodeInfo> nns = Collections.emptyList();
     if (logRollPeriodMs >= 0) {
       try {
@@ -172,8 +173,9 @@ public class EditLogTailer {
           DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY + " is negative.");
     }
     
-    sleepTimeMs = conf.getInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY,
-        DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_DEFAULT) * 1000;
+    sleepTimeMs = conf.getTimeDuration(
+        DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY,
+        DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_DEFAULT, TimeUnit.SECONDS) * 1000;
 
     rollEditsTimeoutMs = conf.getInt(
         DFSConfigKeys.DFS_HA_TAILEDITS_ROLLEDITS_TIMEOUT_KEY,

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -34,6 +34,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
@@ -742,9 +743,10 @@ public class DFSAdmin extends FsShell {
     long timeWindow = 0;
     long txGap = 0;
     if (argv.length > 1 && "-beforeShutdown".equals(argv[1])) {
-      final long checkpointPeriod = dfsConf.getLong(
+      final long checkpointPeriod = dfsConf.getTimeDuration(
           DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY,
-          DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
+          DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT,
+          TimeUnit.SECONDS);
       final long checkpointTxnCount = dfsConf.getLong(
           DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
           DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);

+ 35 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -637,8 +637,12 @@
 </property>
 
 <property>
-  <name>dfs.blockreport.initialDelay</name>  <value>0</value>
-  <description>Delay for first block report in seconds.</description>
+  <name>dfs.blockreport.initialDelay</name>
+  <value>0</value>
+  <description>
+    Delay for first block report in seconds. Support multiple time unit
+    suffix(case insensitive), as described in dfs.heartbeat.interval.
+  </description>
 </property>
 
 <property>
@@ -681,6 +685,8 @@
   <value>21600</value>
   <description>Interval in seconds for Datanode to scan data directories and
   reconcile the difference between blocks in memory and on the disk.
+  Support multiple time unit suffix(case insensitive), as described
+  in dfs.heartbeat.interval.
   </description>
 </property>
 
@@ -715,7 +721,13 @@
 <property>
   <name>dfs.heartbeat.interval</name>
   <value>3</value>
-  <description>Determines datanode heartbeat interval in seconds.</description>
+  <description>
+    Determines datanode heartbeat interval in seconds.
+    Can use the following suffix (case insensitive):
+    ms(millis), s(sec), m(min), h(hour), d(day)
+    to specify the time (such as 2s, 2m, 1h, etc.).
+    Or provide complete number in seconds (such as 30 for 30 seconds).
+  </description>
 </property>
 
 <property>
@@ -920,7 +932,9 @@
   <name>dfs.namenode.decommission.interval</name>
   <value>30</value>
   <description>Namenode periodicity in seconds to check if decommission is 
-  complete.</description>
+  complete. Support multiple time unit suffix(case insensitive), as described
+  in dfs.heartbeat.interval.
+  </description>
 </property>
 
 <property>
@@ -949,7 +963,9 @@
   <name>dfs.namenode.replication.interval</name>
   <value>3</value>
   <description>The periodicity in seconds with which the namenode computes 
-  replication work for datanodes. </description>
+  replication work for datanodes. Support multiple time unit suffix(case insensitive),
+  as described in dfs.heartbeat.interval.
+  </description>
 </property>
 
 <property>
@@ -1044,7 +1060,10 @@
 <property>
   <name>dfs.namenode.checkpoint.period</name>
   <value>3600</value>
-  <description>The number of seconds between two periodic checkpoints.
+  <description>
+    The number of seconds between two periodic checkpoints.
+    Support multiple time unit suffix(case insensitive), as described
+    in dfs.heartbeat.interval.
   </description>
 </property>
 
@@ -1062,7 +1081,8 @@
   <value>60</value>
   <description>The SecondaryNameNode and CheckpointNode will poll the NameNode
   every 'dfs.namenode.checkpoint.check.period' seconds to query the number
-  of uncheckpointed transactions.
+  of uncheckpointed transactions. Support multiple time unit suffix(case insensitive),
+  as described in dfs.heartbeat.interval.
   </description>
 </property>
 
@@ -1408,6 +1428,8 @@
     the datanode dead and invoking the normal recovery mechanisms.
     The notification is sent by a datanode when it is being shutdown
     using the shutdownDatanode admin command with the upgrade option.
+    Support multiple time unit suffix(case insensitive), as described
+    in dfs.heartbeat.interval.
   </description>
 </property>
 
@@ -1475,6 +1497,8 @@
     log segments, the StandbyNode will only be as up-to-date as how
     often the logs are rolled. Note that failover triggers a log roll
     so the StandbyNode will be up to date before it becomes active.
+    Support multiple time unit suffix(case insensitive), as described
+    in dfs.heartbeat.interval.
   </description>
 </property>
 
@@ -1484,6 +1508,8 @@
   <description>
     How often, in seconds, the StandbyNode should check for new
     finalized log segments in the shared edits log.
+    Support multiple time unit suffix(case insensitive), as described
+    in dfs.heartbeat.interval.
   </description>
 </property>
 
@@ -2918,6 +2944,8 @@
     received request. Setting this to 0 fails requests right away if the
     datanode is not yet registered with the namenode. This wait time
     reduces initial request failures after datanode restart.
+    Support multiple time unit suffix(case insensitive), as described
+    in dfs.heartbeat.interval.
   </description>
 </property>