|
@@ -22,6 +22,8 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeAdminBackoffMonitor;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeAdminMonitorInterface;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.After;
|
|
@@ -62,6 +64,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KE
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AVOID_SLOW_DATANODE_FOR_READ_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_LIMIT;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK;
|
|
|
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE_DEFAULT;
|
|
|
|
|
|
public class TestNameNodeReconfigure {
|
|
@@ -567,6 +571,87 @@ public class TestNameNodeReconfigure {
|
|
|
return containReport;
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testReconfigureDecommissionBackoffMonitorParameters()
|
|
|
+ throws ReconfigurationException, IOException {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ conf.setClass(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MONITOR_CLASS,
|
|
|
+ DatanodeAdminBackoffMonitor.class, DatanodeAdminMonitorInterface.class);
|
|
|
+ int defaultPendingRepLimit = 1000;
|
|
|
+ conf.setInt(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_LIMIT, defaultPendingRepLimit);
|
|
|
+ int defaultBlocksPerLock = 1000;
|
|
|
+ conf.setInt(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK,
|
|
|
+ defaultBlocksPerLock);
|
|
|
+
|
|
|
+ try (MiniDFSCluster newCluster = new MiniDFSCluster.Builder(conf).build()) {
|
|
|
+ newCluster.waitActive();
|
|
|
+ final NameNode nameNode = newCluster.getNameNode();
|
|
|
+ final DatanodeManager datanodeManager = nameNode.namesystem
|
|
|
+ .getBlockManager().getDatanodeManager();
|
|
|
+
|
|
|
+ // verify defaultPendingRepLimit.
|
|
|
+ assertEquals(datanodeManager.getDatanodeAdminManager().getPendingRepLimit(),
|
|
|
+ defaultPendingRepLimit);
|
|
|
+
|
|
|
+ // try invalid pendingRepLimit.
|
|
|
+ try {
|
|
|
+ nameNode.reconfigureProperty(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_LIMIT,
|
|
|
+ "non-numeric");
|
|
|
+ fail("Should not reach here");
|
|
|
+ } catch (ReconfigurationException e) {
|
|
|
+ assertEquals("Could not change property " +
|
|
|
+ "dfs.namenode.decommission.backoff.monitor.pending.limit from '" +
|
|
|
+ defaultPendingRepLimit + "' to 'non-numeric'", e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ nameNode.reconfigureProperty(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_LIMIT,
|
|
|
+ "-1");
|
|
|
+ fail("Should not reach here");
|
|
|
+ } catch (ReconfigurationException e) {
|
|
|
+ assertEquals("Could not change property " +
|
|
|
+ "dfs.namenode.decommission.backoff.monitor.pending.limit from '" +
|
|
|
+ defaultPendingRepLimit + "' to '-1'", e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ // try correct pendingRepLimit.
|
|
|
+ nameNode.reconfigureProperty(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_LIMIT,
|
|
|
+ "20000");
|
|
|
+ assertEquals(datanodeManager.getDatanodeAdminManager().getPendingRepLimit(), 20000);
|
|
|
+
|
|
|
+ // verify defaultBlocksPerLock.
|
|
|
+ assertEquals(datanodeManager.getDatanodeAdminManager().getBlocksPerLock(),
|
|
|
+ defaultBlocksPerLock);
|
|
|
+
|
|
|
+ // try invalid blocksPerLock.
|
|
|
+ try {
|
|
|
+ nameNode.reconfigureProperty(
|
|
|
+ DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK,
|
|
|
+ "non-numeric");
|
|
|
+ fail("Should not reach here");
|
|
|
+ } catch (ReconfigurationException e) {
|
|
|
+ assertEquals("Could not change property " +
|
|
|
+ "dfs.namenode.decommission.backoff.monitor.pending.blocks.per.lock from '" +
|
|
|
+ defaultBlocksPerLock + "' to 'non-numeric'", e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ nameNode.reconfigureProperty(
|
|
|
+ DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK, "-1");
|
|
|
+ fail("Should not reach here");
|
|
|
+ } catch (ReconfigurationException e) {
|
|
|
+ assertEquals("Could not change property " +
|
|
|
+ "dfs.namenode.decommission.backoff.monitor.pending.blocks.per.lock from '" +
|
|
|
+ defaultBlocksPerLock + "' to '-1'", e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ // try correct blocksPerLock.
|
|
|
+ nameNode.reconfigureProperty(
|
|
|
+ DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK, "10000");
|
|
|
+ assertEquals(datanodeManager.getDatanodeAdminManager().getBlocksPerLock(), 10000);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@After
|
|
|
public void shutDown() throws IOException {
|
|
|
if (cluster != null) {
|