HDFS-14346. Add better time precision to Configuration#getTimeDuration, allowing return unit and default unit to be specified independently. Contributed by Chao Sun.

This commit is contained in:
Erik Krogen 2019-03-13 09:38:10 -07:00
parent 4fa009989b
commit 66357574ae
9 changed files with 77 additions and 23 deletions

View File

@ -1805,6 +1805,7 @@ public void setTimeDuration(String name, long value, TimeUnit unit) {
* Return time duration in the given time unit. Valid units are encoded in * Return time duration in the given time unit. Valid units are encoded in
* properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
* (ms), seconds (s), minutes (m), hours (h), and days (d). * (ms), seconds (s), minutes (m), hours (h), and days (d).
*
* @param name Property name * @param name Property name
* @param defaultValue Value returned if no mapping exists. * @param defaultValue Value returned if no mapping exists.
* @param unit Unit to convert the stored property, if it exists. * @param unit Unit to convert the stored property, if it exists.
@ -1813,20 +1814,44 @@ public void setTimeDuration(String name, long value, TimeUnit unit) {
* @return time duration in given time unit * @return time duration in given time unit
*/ */
public long getTimeDuration(String name, long defaultValue, TimeUnit unit) { public long getTimeDuration(String name, long defaultValue, TimeUnit unit) {
return getTimeDuration(name, defaultValue, unit, unit);
}
public long getTimeDuration(String name, String defaultValue, TimeUnit unit) {
return getTimeDuration(name, defaultValue, unit, unit);
}
/**
* Return time duration in the given time unit. Valid units are encoded in
* properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
* (ms), seconds (s), minutes (m), hours (h), and days (d). If no unit is
* provided, the default unit is applied.
*
* @param name Property name
* @param defaultValue Value returned if no mapping exists.
* @param defaultUnit Default time unit if no valid suffix is provided.
* @param returnUnit The unit used for the returned value.
* @throws NumberFormatException If the property stripped of its unit is not
* a number
* @return time duration in given time unit
*/
public long getTimeDuration(String name, long defaultValue,
TimeUnit defaultUnit, TimeUnit returnUnit) {
String vStr = get(name); String vStr = get(name);
if (null == vStr) { if (null == vStr) {
return defaultValue; return defaultValue;
} else { } else {
return getTimeDurationHelper(name, vStr, unit); return getTimeDurationHelper(name, vStr, defaultUnit, returnUnit);
} }
} }
public long getTimeDuration(String name, String defaultValue, TimeUnit unit) { public long getTimeDuration(String name, String defaultValue,
TimeUnit defaultUnit, TimeUnit returnUnit) {
String vStr = get(name); String vStr = get(name);
if (null == vStr) { if (null == vStr) {
return getTimeDurationHelper(name, defaultValue, unit); return getTimeDurationHelper(name, defaultValue, defaultUnit, returnUnit);
} else { } else {
return getTimeDurationHelper(name, vStr, unit); return getTimeDurationHelper(name, vStr, defaultUnit, returnUnit);
} }
} }
@ -1834,26 +1859,43 @@ public long getTimeDuration(String name, String defaultValue, TimeUnit unit) {
* Return time duration in the given time unit. Valid units are encoded in * Return time duration in the given time unit. Valid units are encoded in
* properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
* (ms), seconds (s), minutes (m), hours (h), and days (d). * (ms), seconds (s), minutes (m), hours (h), and days (d).
*
* @param name Property name * @param name Property name
* @param vStr The string value with time unit suffix to be converted. * @param vStr The string value with time unit suffix to be converted.
* @param unit Unit to convert the stored property, if it exists. * @param unit Unit to convert the stored property, if it exists.
*/ */
public long getTimeDurationHelper(String name, String vStr, TimeUnit unit) { public long getTimeDurationHelper(String name, String vStr, TimeUnit unit) {
return getTimeDurationHelper(name, vStr, unit, unit);
}
/**
* Return time duration in the given time unit. Valid units are encoded in
* properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
* (ms), seconds (s), minutes (m), hours (h), and days (d).
*
* @param name Property name
* @param vStr The string value with time unit suffix to be converted.
* @param defaultUnit Unit to convert the stored property, if it exists.
* @param returnUnit Unit for the returned value.
*/
private long getTimeDurationHelper(String name, String vStr,
TimeUnit defaultUnit, TimeUnit returnUnit) {
vStr = vStr.trim(); vStr = vStr.trim();
vStr = StringUtils.toLowerCase(vStr); vStr = StringUtils.toLowerCase(vStr);
ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr); ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr);
if (null == vUnit) { if (null == vUnit) {
logDeprecation("No unit for " + name + "(" + vStr + ") assuming " + unit); logDeprecation("No unit for " + name + "(" + vStr + ") assuming " +
vUnit = ParsedTimeDuration.unitFor(unit); defaultUnit);
vUnit = ParsedTimeDuration.unitFor(defaultUnit);
} else { } else {
vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix())); vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix()));
} }
long raw = Long.parseLong(vStr); long raw = Long.parseLong(vStr);
long converted = unit.convert(raw, vUnit.unit()); long converted = returnUnit.convert(raw, vUnit.unit());
if (vUnit.unit().convert(converted, unit) < raw) { if (vUnit.unit().convert(converted, returnUnit) < raw) {
logDeprecation("Possible loss of precision converting " + vStr logDeprecation("Possible loss of precision converting " + vStr
+ vUnit.suffix() + " to " + unit + " for " + name); + vUnit.suffix() + " to " + returnUnit + " for " + name);
} }
return converted; return converted;
} }

View File

@ -1404,7 +1404,10 @@ public void testTimeDuration() {
conf.setTimeDuration("test.time.a", 7L, SECONDS); conf.setTimeDuration("test.time.a", 7L, SECONDS);
assertEquals("7s", conf.get("test.time.a")); assertEquals("7s", conf.get("test.time.a"));
assertEquals(0L, conf.getTimeDuration("test.time.a", 30, MINUTES)); assertEquals(0L, conf.getTimeDuration("test.time.a", 30, MINUTES));
assertEquals(0L, conf.getTimeDuration("test.time.a", 30, SECONDS, MINUTES));
assertEquals(7L, conf.getTimeDuration("test.time.a", 30, SECONDS)); assertEquals(7L, conf.getTimeDuration("test.time.a", 30, SECONDS));
assertEquals(7L,
conf.getTimeDuration("test.time.a", 30, MILLISECONDS, SECONDS));
assertEquals(7000L, conf.getTimeDuration("test.time.a", 30, MILLISECONDS)); assertEquals(7000L, conf.getTimeDuration("test.time.a", 30, MILLISECONDS));
assertEquals(7000000L, assertEquals(7000000L,
conf.getTimeDuration("test.time.a", 30, MICROSECONDS)); conf.getTimeDuration("test.time.a", 30, MICROSECONDS));
@ -1421,6 +1424,8 @@ public void testTimeDuration() {
assertEquals(30L, conf.getTimeDuration("test.time.X", 30, SECONDS)); assertEquals(30L, conf.getTimeDuration("test.time.X", 30, SECONDS));
conf.set("test.time.X", "30"); conf.set("test.time.X", "30");
assertEquals(30L, conf.getTimeDuration("test.time.X", 40, SECONDS)); assertEquals(30L, conf.getTimeDuration("test.time.X", 40, SECONDS));
assertEquals(30000L,
conf.getTimeDuration("test.time.X", 40, SECONDS, MILLISECONDS));
assertEquals(10L, conf.getTimeDuration("test.time.c", "10", SECONDS)); assertEquals(10L, conf.getTimeDuration("test.time.c", "10", SECONDS));
assertEquals(30L, conf.getTimeDuration("test.time.c", "30s", SECONDS)); assertEquals(30L, conf.getTimeDuration("test.time.c", "30s", SECONDS));
assertEquals(120L, conf.getTimeDuration("test.time.c", "2m", SECONDS)); assertEquals(120L, conf.getTimeDuration("test.time.c", "2m", SECONDS));

View File

@ -248,7 +248,7 @@ public DfsClientConf(Configuration conf) {
datanodeRestartTimeout = conf.getTimeDuration( datanodeRestartTimeout = conf.getTimeDuration(
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY, DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT, DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT,
TimeUnit.SECONDS) * 1000; TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
slowIoWarningThresholdMs = conf.getLong( slowIoWarningThresholdMs = conf.getLong(
DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY, DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT); DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);

View File

@ -677,11 +677,11 @@ static int run(Collection<URI> namenodes, final BalancerParameters p,
final long sleeptime = final long sleeptime =
conf.getTimeDuration(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, conf.getTimeDuration(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT,
TimeUnit.SECONDS) * 2000 + TimeUnit.SECONDS, TimeUnit.MILLISECONDS) * 2 +
conf.getTimeDuration( conf.getTimeDuration(
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
TimeUnit.SECONDS) * 1000; TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
LOG.info("namenodes = " + namenodes); LOG.info("namenodes = " + namenodes);
LOG.info("parameters = " + p); LOG.info("parameters = " + p);
LOG.info("included nodes = " + p.getIncludedNodes()); LOG.info("included nodes = " + p.getIncludedNodes());

View File

@ -520,7 +520,7 @@ public BlockManager(final Namesystem namesystem, boolean haEnabled,
this.redundancyRecheckIntervalMs = conf.getTimeDuration( this.redundancyRecheckIntervalMs = conf.getTimeDuration(
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
TimeUnit.SECONDS) * 1000; TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
this.storageInfoDefragmentInterval = this.storageInfoDefragmentInterval =
conf.getLong( conf.getLong(

View File

@ -124,7 +124,8 @@ public void initialize(Configuration conf, FSClusterStats stats,
this.host2datanodeMap = host2datanodeMap; this.host2datanodeMap = host2datanodeMap;
this.heartbeatInterval = conf.getTimeDuration( this.heartbeatInterval = conf.getTimeDuration(
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000; DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT,
TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
this.tolerateHeartbeatMultiplier = conf.getInt( this.tolerateHeartbeatMultiplier = conf.getInt(
DFSConfigKeys.DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_KEY, DFSConfigKeys.DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_KEY,
DFSConfigKeys.DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_DEFAULT); DFSConfigKeys.DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_DEFAULT);

View File

@ -199,7 +199,8 @@ public DNConf(final Configurable dn) {
long initBRDelay = getConf().getTimeDuration( long initBRDelay = getConf().getTimeDuration(
DFS_BLOCKREPORT_INITIAL_DELAY_KEY, DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT, TimeUnit.SECONDS) * 1000L; DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT,
TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
if (initBRDelay >= blockReportInterval) { if (initBRDelay >= blockReportInterval) {
initBRDelay = 0; initBRDelay = 0;
DataNode.LOG.info("dfs.blockreport.initialDelay is " DataNode.LOG.info("dfs.blockreport.initialDelay is "
@ -209,11 +210,13 @@ public DNConf(final Configurable dn) {
initialBlockReportDelayMs = initBRDelay; initialBlockReportDelayMs = initBRDelay;
heartBeatInterval = getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY, heartBeatInterval = getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000L; DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS,
TimeUnit.MILLISECONDS);
long confLifelineIntervalMs = long confLifelineIntervalMs =
getConf().getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY, getConf().getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY,
3 * getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY, 3 * getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS)) * 1000L; DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS,
TimeUnit.MILLISECONDS));
if (confLifelineIntervalMs <= heartBeatInterval) { if (confLifelineIntervalMs <= heartBeatInterval) {
confLifelineIntervalMs = 3 * heartBeatInterval; confLifelineIntervalMs = 3 * heartBeatInterval;
DataNode.LOG.warn( DataNode.LOG.warn(

View File

@ -631,11 +631,11 @@ static int run(Map<URI, List<Path>> namenodes, Configuration conf)
final long sleeptime = final long sleeptime =
conf.getTimeDuration(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, conf.getTimeDuration(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT,
TimeUnit.SECONDS) * 2000 + TimeUnit.SECONDS, TimeUnit.MILLISECONDS) * 2 +
conf.getTimeDuration( conf.getTimeDuration(
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
TimeUnit.SECONDS) * 1000; TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
AtomicInteger retryCount = new AtomicInteger(0); AtomicInteger retryCount = new AtomicInteger(0);
// TODO: Need to limit the size of the pinned blocks to limit memory usage // TODO: Need to limit the size of the pinned blocks to limit memory usage
Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks = new HashMap<>(); Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks = new HashMap<>();

View File

@ -176,7 +176,8 @@ public EditLogTailer(FSNamesystem namesystem, Configuration conf) {
logRollPeriodMs = conf.getTimeDuration( logRollPeriodMs = conf.getTimeDuration(
DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT, TimeUnit.SECONDS) * 1000; DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT,
TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
List<RemoteNameNodeInfo> nns = Collections.emptyList(); List<RemoteNameNodeInfo> nns = Collections.emptyList();
if (logRollPeriodMs >= 0) { if (logRollPeriodMs >= 0) {
try { try {
@ -203,11 +204,13 @@ public EditLogTailer(FSNamesystem namesystem, Configuration conf) {
sleepTimeMs = conf.getTimeDuration( sleepTimeMs = conf.getTimeDuration(
DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY,
DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_DEFAULT, TimeUnit.SECONDS) * 1000; DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_DEFAULT,
TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
rollEditsTimeoutMs = conf.getInt( rollEditsTimeoutMs = conf.getTimeDuration(
DFSConfigKeys.DFS_HA_TAILEDITS_ROLLEDITS_TIMEOUT_KEY, DFSConfigKeys.DFS_HA_TAILEDITS_ROLLEDITS_TIMEOUT_KEY,
DFSConfigKeys.DFS_HA_TAILEDITS_ROLLEDITS_TIMEOUT_DEFAULT) * 1000; DFSConfigKeys.DFS_HA_TAILEDITS_ROLLEDITS_TIMEOUT_DEFAULT,
TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
rollEditsRpcExecutor = Executors.newSingleThreadExecutor( rollEditsRpcExecutor = Executors.newSingleThreadExecutor(
new ThreadFactoryBuilder().setDaemon(true).build()); new ThreadFactoryBuilder().setDaemon(true).build());