|
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Joiner;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import com.google.common.collect.Lists;
|
|
|
+import com.google.common.collect.Sets;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -97,8 +98,8 @@ import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
-import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
+import java.util.TreeSet;
|
|
|
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
@@ -155,6 +156,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHEC
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.FS_PROTECTED_DIRECTORIES;
|
|
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE;
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_NAMESPACE;
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE_DEFAULT;
|
|
|
|
|
|
/**********************************************************
|
|
|
* NameNode serves as both directory namespace manager and
|
|
@@ -274,13 +278,15 @@ public class NameNode extends ReconfigurableBase implements
|
|
|
DFS_HA_AUTO_FAILOVER_ENABLED_KEY
|
|
|
};
|
|
|
|
|
|
+ private String ipcClientRPCBackoffEnable;
|
|
|
+
|
|
|
/** A list of property that are reconfigurable at runtime. */
|
|
|
- static final List<String> RECONFIGURABLE_PROPERTIES = Collections
|
|
|
- .unmodifiableList(Arrays
|
|
|
- .asList(DFS_HEARTBEAT_INTERVAL_KEY,
|
|
|
- DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
- FS_PROTECTED_DIRECTORIES,
|
|
|
- HADOOP_CALLER_CONTEXT_ENABLED_KEY));
|
|
|
+ private final TreeSet<String> reconfigurableProperties = Sets
|
|
|
+ .newTreeSet(Lists.newArrayList(
|
|
|
+ DFS_HEARTBEAT_INTERVAL_KEY,
|
|
|
+ DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
+ FS_PROTECTED_DIRECTORIES,
|
|
|
+ HADOOP_CALLER_CONTEXT_ENABLED_KEY));
|
|
|
|
|
|
private static final String USAGE = "Usage: hdfs namenode ["
|
|
|
+ StartupOption.BACKUP.getName() + "] | \n\t["
|
|
@@ -702,6 +708,9 @@ public class NameNode extends ReconfigurableBase implements
|
|
|
loadNamesystem(conf);
|
|
|
|
|
|
rpcServer = createRpcServer(conf);
|
|
|
+
|
|
|
+ initReconfigurableBackoffKey();
|
|
|
+
|
|
|
if (clientNamenodeAddress == null) {
|
|
|
// This is expected for MiniDFSCluster. Set it now using
|
|
|
// the RPC server's bind address.
|
|
@@ -719,6 +728,18 @@ public class NameNode extends ReconfigurableBase implements
|
|
|
startMetricsLogger(conf);
|
|
|
}
|
|
|
|
|
|
+ private void initReconfigurableBackoffKey() {
|
|
|
+ ipcClientRPCBackoffEnable = buildBackoffEnableKey(rpcServer
|
|
|
+ .getClientRpcServer().getPort());
|
|
|
+ reconfigurableProperties.add(ipcClientRPCBackoffEnable);
|
|
|
+ }
|
|
|
+
|
|
|
+ static String buildBackoffEnableKey(final int port) {
|
|
|
+ // format used to construct backoff enable key, e.g. ipc.8020.backoff.enable
|
|
|
+ String format = "%s.%d.%s";
|
|
|
+ return String.format(format, IPC_NAMESPACE, port, IPC_BACKOFF_ENABLE);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Start a timer to periodically write NameNode metrics to the log
|
|
|
* file. This behavior can be disabled by configuration.
|
|
@@ -1956,7 +1977,7 @@ public class NameNode extends ReconfigurableBase implements
|
|
|
* */
|
|
|
@Override // ReconfigurableBase
|
|
|
public Collection<String> getReconfigurableProperties() {
|
|
|
- return RECONFIGURABLE_PROPERTIES;
|
|
|
+ return reconfigurableProperties;
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -1968,57 +1989,67 @@ public class NameNode extends ReconfigurableBase implements
|
|
|
final DatanodeManager datanodeManager = namesystem.getBlockManager()
|
|
|
.getDatanodeManager();
|
|
|
|
|
|
- switch (property) {
|
|
|
- case DFS_HEARTBEAT_INTERVAL_KEY:
|
|
|
- namesystem.writeLock();
|
|
|
- try {
|
|
|
- if (newVal == null) {
|
|
|
- // set to default
|
|
|
- datanodeManager.setHeartbeatInterval(DFS_HEARTBEAT_INTERVAL_DEFAULT);
|
|
|
- return String.valueOf(DFS_HEARTBEAT_INTERVAL_DEFAULT);
|
|
|
- } else {
|
|
|
- datanodeManager.setHeartbeatInterval(Long.parseLong(newVal));
|
|
|
- return String.valueOf(datanodeManager.getHeartbeatInterval());
|
|
|
- }
|
|
|
- } catch (NumberFormatException nfe) {
|
|
|
- throw new ReconfigurationException(property, newVal, getConf().get(
|
|
|
- property), nfe);
|
|
|
- } finally {
|
|
|
- namesystem.writeUnlock();
|
|
|
- LOG.info("RECONFIGURE* changed heartbeatInterval to "
|
|
|
- + datanodeManager.getHeartbeatInterval());
|
|
|
- }
|
|
|
- case DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY:
|
|
|
- namesystem.writeLock();
|
|
|
- try {
|
|
|
- if (newVal == null) {
|
|
|
- // set to default
|
|
|
- datanodeManager
|
|
|
- .setHeartbeatRecheckInterval(
|
|
|
- DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT);
|
|
|
- return String
|
|
|
- .valueOf(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT);
|
|
|
- } else {
|
|
|
- datanodeManager.setHeartbeatRecheckInterval(Integer.parseInt(newVal));
|
|
|
- return String.valueOf(datanodeManager.getHeartbeatRecheckInterval());
|
|
|
- }
|
|
|
- } catch (NumberFormatException nfe) {
|
|
|
- throw new ReconfigurationException(property, newVal, getConf().get(
|
|
|
- property), nfe);
|
|
|
- } finally {
|
|
|
- namesystem.writeUnlock();
|
|
|
- LOG.info("RECONFIGURE* changed heartbeatRecheckInterval to "
|
|
|
- + datanodeManager.getHeartbeatRecheckInterval());
|
|
|
- }
|
|
|
- case FS_PROTECTED_DIRECTORIES:
|
|
|
+ if (property.equals(DFS_HEARTBEAT_INTERVAL_KEY)) {
|
|
|
+ return reconfHeartbeatInterval(datanodeManager, property, newVal);
|
|
|
+ } else if (property.equals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY)) {
|
|
|
+ return reconfHeartbeatRecheckInterval(datanodeManager, property, newVal);
|
|
|
+ } else if (property.equals(FS_PROTECTED_DIRECTORIES)) {
|
|
|
return reconfProtectedDirectories(newVal);
|
|
|
- case HADOOP_CALLER_CONTEXT_ENABLED_KEY:
|
|
|
+ } else if (property.equals(HADOOP_CALLER_CONTEXT_ENABLED_KEY)) {
|
|
|
return reconfCallerContextEnabled(newVal);
|
|
|
- default:
|
|
|
- break;
|
|
|
+ } else if (property.equals(ipcClientRPCBackoffEnable)) {
|
|
|
+ return reconfigureIPCBackoffEnabled(newVal);
|
|
|
+ } else {
|
|
|
+ throw new ReconfigurationException(property, newVal, getConf().get(
|
|
|
+ property));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private String reconfHeartbeatInterval(final DatanodeManager datanodeManager,
|
|
|
+ final String property, final String newVal)
|
|
|
+ throws ReconfigurationException {
|
|
|
+ namesystem.writeLock();
|
|
|
+ try {
|
|
|
+ if (newVal == null) {
|
|
|
+ // set to default
|
|
|
+ datanodeManager.setHeartbeatInterval(DFS_HEARTBEAT_INTERVAL_DEFAULT);
|
|
|
+ return String.valueOf(DFS_HEARTBEAT_INTERVAL_DEFAULT);
|
|
|
+ } else {
|
|
|
+ datanodeManager.setHeartbeatInterval(Long.parseLong(newVal));
|
|
|
+ return String.valueOf(datanodeManager.getHeartbeatInterval());
|
|
|
+ }
|
|
|
+ } catch (NumberFormatException nfe) {
|
|
|
+ throw new ReconfigurationException(property, newVal, getConf().get(
|
|
|
+ property), nfe);
|
|
|
+ } finally {
|
|
|
+ namesystem.writeUnlock();
|
|
|
+ LOG.info("RECONFIGURE* changed heartbeatInterval to "
|
|
|
+ + datanodeManager.getHeartbeatInterval());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private String reconfHeartbeatRecheckInterval(
|
|
|
+ final DatanodeManager datanodeManager, final String property,
|
|
|
+ final String newVal) throws ReconfigurationException {
|
|
|
+ namesystem.writeLock();
|
|
|
+ try {
|
|
|
+ if (newVal == null) {
|
|
|
+ // set to default
|
|
|
+ datanodeManager.setHeartbeatRecheckInterval(
|
|
|
+ DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT);
|
|
|
+ return String.valueOf(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT);
|
|
|
+ } else {
|
|
|
+ datanodeManager.setHeartbeatRecheckInterval(Integer.parseInt(newVal));
|
|
|
+ return String.valueOf(datanodeManager.getHeartbeatRecheckInterval());
|
|
|
+ }
|
|
|
+ } catch (NumberFormatException nfe) {
|
|
|
+ throw new ReconfigurationException(property, newVal, getConf().get(
|
|
|
+ property), nfe);
|
|
|
+ } finally {
|
|
|
+ namesystem.writeUnlock();
|
|
|
+ LOG.info("RECONFIGURE* changed heartbeatRecheckInterval to "
|
|
|
+ + datanodeManager.getHeartbeatRecheckInterval());
|
|
|
}
|
|
|
- throw new ReconfigurationException(property, newVal, getConf()
|
|
|
- .get(property));
|
|
|
}
|
|
|
|
|
|
private String reconfProtectedDirectories(String newVal) {
|
|
@@ -2036,6 +2067,18 @@ public class NameNode extends ReconfigurableBase implements
|
|
|
return Boolean.toString(callerContextEnabled);
|
|
|
}
|
|
|
|
|
|
+ String reconfigureIPCBackoffEnabled(String newVal) {
|
|
|
+ boolean clientBackoffEnabled;
|
|
|
+ if (newVal == null) {
|
|
|
+ clientBackoffEnabled = IPC_BACKOFF_ENABLE_DEFAULT;
|
|
|
+ } else {
|
|
|
+ clientBackoffEnabled = Boolean.parseBoolean(newVal);
|
|
|
+ }
|
|
|
+ rpcServer.getClientRpcServer()
|
|
|
+ .setClientBackoffEnabled(clientBackoffEnabled);
|
|
|
+ return Boolean.toString(clientBackoffEnabled);
|
|
|
+ }
|
|
|
+
|
|
|
@Override // ReconfigurableBase
|
|
|
protected Configuration getNewConf() {
|
|
|
return new HdfsConfiguration();
|