HDFS-11368. Erasure Coding: Deprecate replication-related config keys. Contributed by Rakesh R.

This commit is contained in:
Andrew Wang 2016-11-23 16:42:06 -08:00
parent dd98a8005a
commit de4894936a
52 changed files with 216 additions and 158 deletions

View File

@ -43,8 +43,11 @@ The following table lists the configuration property names that are deprecated i
| dfs.permissions | dfs.permissions.enabled | | dfs.permissions | dfs.permissions.enabled |
| dfs.permissions.supergroup | dfs.permissions.superusergroup | | dfs.permissions.supergroup | dfs.permissions.superusergroup |
| dfs.read.prefetch.size | dfs.client.read.prefetch.size | | dfs.read.prefetch.size | dfs.client.read.prefetch.size |
| dfs.replication.considerLoad | dfs.namenode.replication.considerLoad | | dfs.replication.considerLoad | dfs.namenode.redundancy.considerLoad |
| dfs.replication.interval | dfs.namenode.replication.interval | | dfs.namenode.replication.considerLoad | dfs.namenode.redundancy.considerLoad |
| dfs.namenode.replication.considerLoad.factor | dfs.namenode.redundancy.considerLoad.factor |
| dfs.replication.interval | dfs.namenode.redundancy.interval |
| dfs.namenode.replication.interval | dfs.namenode.redundancy.interval |
| dfs.replication.min | dfs.namenode.replication.min | | dfs.replication.min | dfs.namenode.replication.min |
| dfs.replication.pending.timeout.sec | dfs.namenode.reconstruction.pending.timeout-sec | | dfs.replication.pending.timeout.sec | dfs.namenode.reconstruction.pending.timeout-sec |
| dfs.namenode.replication.pending.timeout-sec | dfs.namenode.reconstruction.pending.timeout-sec | | dfs.namenode.replication.pending.timeout-sec | dfs.namenode.reconstruction.pending.timeout-sec |

View File

@ -115,9 +115,15 @@ public class HdfsConfiguration extends Configuration {
new DeprecationDelta("dfs.access.time.precision", new DeprecationDelta("dfs.access.time.precision",
DeprecatedKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY), DeprecatedKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY),
new DeprecationDelta("dfs.replication.considerLoad", new DeprecationDelta("dfs.replication.considerLoad",
DeprecatedKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY), DeprecatedKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY),
new DeprecationDelta("dfs.namenode.replication.considerLoad",
DeprecatedKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY),
new DeprecationDelta("dfs.namenode.replication.considerLoad.factor",
DeprecatedKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR),
new DeprecationDelta("dfs.replication.interval", new DeprecationDelta("dfs.replication.interval",
DeprecatedKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY), DeprecatedKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY),
new DeprecationDelta("dfs.namenode.replication.interval",
DeprecatedKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY),
new DeprecationDelta("dfs.replication.min", new DeprecationDelta("dfs.replication.min",
DeprecatedKeys.DFS_NAMENODE_REPLICATION_MIN_KEY), DeprecatedKeys.DFS_NAMENODE_REPLICATION_MIN_KEY),
new DeprecationDelta("dfs.replication.pending.timeout.sec", new DeprecationDelta("dfs.replication.pending.timeout.sec",

View File

@ -204,10 +204,12 @@ public interface HdfsClientConfigKeys {
String DFS_METRICS_SESSION_ID_KEY = "dfs.metrics.session-id"; String DFS_METRICS_SESSION_ID_KEY = "dfs.metrics.session-id";
String DFS_NAMENODE_ACCESSTIME_PRECISION_KEY = String DFS_NAMENODE_ACCESSTIME_PRECISION_KEY =
"dfs.namenode.accesstime.precision"; "dfs.namenode.accesstime.precision";
String DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY = String DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY =
"dfs.namenode.replication.considerLoad"; "dfs.namenode.redundancy.considerLoad";
String DFS_NAMENODE_REPLICATION_INTERVAL_KEY = String DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR =
"dfs.namenode.replication.interval"; "dfs.namenode.redundancy.considerLoad.factor";
String DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY =
"dfs.namenode.redundancy.interval.seconds";
String DFS_NAMENODE_REPLICATION_MIN_KEY = "dfs.namenode.replication.min"; String DFS_NAMENODE_REPLICATION_MIN_KEY = "dfs.namenode.replication.min";
String DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY = String DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY =
"dfs.namenode.reconstruction.pending.timeout-sec"; "dfs.namenode.reconstruction.pending.timeout-sec";

View File

@ -194,16 +194,17 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_ACCESSTIME_PRECISION_KEY = public static final String DFS_NAMENODE_ACCESSTIME_PRECISION_KEY =
HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY; HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
public static final long DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT = 3600000; public static final long DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT = 3600000;
public static final String DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY = public static final String DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY =
HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY; HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY;
public static final boolean DFS_NAMENODE_REPLICATION_CONSIDERLOAD_DEFAULT = true; public static final boolean DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_DEFAULT =
public static final String DFS_NAMENODE_REPLICATION_CONSIDERLOAD_FACTOR = true;
"dfs.namenode.replication.considerLoad.factor"; public static final String DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR =
"dfs.namenode.redundancy.considerLoad.factor";
public static final double public static final double
DFS_NAMENODE_REPLICATION_CONSIDERLOAD_FACTOR_DEFAULT = 2.0; DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR_DEFAULT = 2.0;
public static final String DFS_NAMENODE_REPLICATION_INTERVAL_KEY = public static final String DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY =
HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY; HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY;
public static final int DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT = 3; public static final int DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT = 3;
public static final String DFS_NAMENODE_REPLICATION_MIN_KEY = public static final String DFS_NAMENODE_REPLICATION_MIN_KEY =
HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_MIN_KEY; HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_MIN_KEY;
public static final int DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1; public static final int DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1;

View File

@ -673,8 +673,8 @@ public class Balancer {
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT,
TimeUnit.SECONDS) * 2000 + TimeUnit.SECONDS) * 2000 +
conf.getTimeDuration( conf.getTimeDuration(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
TimeUnit.SECONDS) * 1000; TimeUnit.SECONDS) * 1000;
LOG.info("namenodes = " + namenodes); LOG.info("namenodes = " + namenodes);
LOG.info("parameters = " + p); LOG.info("parameters = " + p);

View File

@ -230,8 +230,11 @@ public class BlockManager implements BlockStatsMXBean {
return pendingReconstruction.getNumTimedOuts(); return pendingReconstruction.getNumTimedOuts();
} }
/**replicationRecheckInterval is how often namenode checks for new replication work*/ /**
private final long replicationRecheckInterval; * redundancyRecheckInterval is how often namenode checks for new
* reconstruction work.
*/
private final long redundancyRecheckIntervalMs;
/** How often to check and the limit for the storageinfo efficiency. */ /** How often to check and the limit for the storageinfo efficiency. */
private final long storageInfoDefragmentInterval; private final long storageInfoDefragmentInterval;
@ -244,8 +247,8 @@ public class BlockManager implements BlockStatsMXBean {
*/ */
final BlocksMap blocksMap; final BlocksMap blocksMap;
/** Replication thread. */ /** Redundancy thread. */
final Daemon replicationThread = new Daemon(new ReplicationMonitor()); private final Daemon redundancyThread = new Daemon(new RedundancyMonitor());
/** StorageInfoDefragmenter thread. */ /** StorageInfoDefragmenter thread. */
private final Daemon storageInfoDefragmenterThread = private final Daemon storageInfoDefragmenterThread =
@ -435,10 +438,10 @@ public class BlockManager implements BlockStatsMXBean {
this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf); this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf);
this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf); this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
this.replicationRecheckInterval = this.redundancyRecheckIntervalMs = conf.getTimeDuration(
conf.getTimeDuration(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
TimeUnit.SECONDS) * 1000L; TimeUnit.SECONDS) * 1000;
this.storageInfoDefragmentInterval = this.storageInfoDefragmentInterval =
conf.getLong( conf.getLong(
@ -493,7 +496,8 @@ public class BlockManager implements BlockStatsMXBean {
LOG.info("maxReplication = " + maxReplication); LOG.info("maxReplication = " + maxReplication);
LOG.info("minReplication = " + minReplication); LOG.info("minReplication = " + minReplication);
LOG.info("maxReplicationStreams = " + maxReplicationStreams); LOG.info("maxReplicationStreams = " + maxReplicationStreams);
LOG.info("replicationRecheckInterval = " + replicationRecheckInterval); LOG.info("redundancyRecheckInterval = " + redundancyRecheckIntervalMs +
"ms");
LOG.info("encryptDataTransfer = " + encryptDataTransfer); LOG.info("encryptDataTransfer = " + encryptDataTransfer);
LOG.info("maxNumBlocksToLog = " + maxNumBlocksToLog); LOG.info("maxNumBlocksToLog = " + maxNumBlocksToLog);
} }
@ -586,7 +590,7 @@ public class BlockManager implements BlockStatsMXBean {
return blockTokenSecretManager; return blockTokenSecretManager;
} }
/** Allow silent termination of replication monitor for testing */ /** Allow silent termination of redundancy monitor for testing. */
@VisibleForTesting @VisibleForTesting
void enableRMTerminationForTesting() { void enableRMTerminationForTesting() {
checkNSRunning = false; checkNSRunning = false;
@ -604,8 +608,8 @@ public class BlockManager implements BlockStatsMXBean {
public void activate(Configuration conf, long blockTotal) { public void activate(Configuration conf, long blockTotal) {
pendingReconstruction.start(); pendingReconstruction.start();
datanodeManager.activate(conf); datanodeManager.activate(conf);
this.replicationThread.setName("ReplicationMonitor"); this.redundancyThread.setName("RedundancyMonitor");
this.replicationThread.start(); this.redundancyThread.start();
storageInfoDefragmenterThread.setName("StorageInfoMonitor"); storageInfoDefragmenterThread.setName("StorageInfoMonitor");
storageInfoDefragmenterThread.start(); storageInfoDefragmenterThread.start();
this.blockReportThread.start(); this.blockReportThread.start();
@ -616,10 +620,10 @@ public class BlockManager implements BlockStatsMXBean {
public void close() { public void close() {
bmSafeMode.close(); bmSafeMode.close();
try { try {
replicationThread.interrupt(); redundancyThread.interrupt();
storageInfoDefragmenterThread.interrupt(); storageInfoDefragmenterThread.interrupt();
blockReportThread.interrupt(); blockReportThread.interrupt();
replicationThread.join(3000); redundancyThread.join(3000);
storageInfoDefragmenterThread.join(3000); storageInfoDefragmenterThread.join(3000);
blockReportThread.join(3000); blockReportThread.join(3000);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
@ -880,7 +884,7 @@ public class BlockManager implements BlockStatsMXBean {
/** /**
* If IBR is not sent from expected locations yet, add the datanodes to * If IBR is not sent from expected locations yet, add the datanodes to
* pendingReconstruction in order to keep ReplicationMonitor from scheduling * pendingReconstruction in order to keep RedundancyMonitor from scheduling
* the block. * the block.
*/ */
public void addExpectedReplicasToPending(BlockInfo blk) { public void addExpectedReplicasToPending(BlockInfo blk) {
@ -1884,7 +1888,7 @@ public class BlockManager implements BlockStatsMXBean {
if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) { if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
neededReconstruction.remove(block, priority); neededReconstruction.remove(block, priority);
rw.resetTargets(); rw.resetTargets();
blockLog.debug("BLOCK* Removing {} from neededReplications as" + blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
" it has enough replicas", block); " it has enough replicas", block);
return false; return false;
} }
@ -1910,8 +1914,8 @@ public class BlockManager implements BlockStatsMXBean {
// reconstructions that fail after an appropriate amount of time. // reconstructions that fail after an appropriate amount of time.
pendingReconstruction.increment(block, pendingReconstruction.increment(block,
DatanodeStorageInfo.toDatanodeDescriptors(targets)); DatanodeStorageInfo.toDatanodeDescriptors(targets));
blockLog.debug("BLOCK* block {} is moved from neededReplications to " blockLog.debug("BLOCK* block {} is moved from neededReconstruction to "
+ "pendingReplications", block); + "pendingReconstruction", block);
int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum; int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
// remove from neededReconstruction // remove from neededReconstruction
@ -4298,32 +4302,32 @@ public class BlockManager implements BlockStatsMXBean {
/** /**
* Periodically calls computeBlockRecoveryWork(). * Periodically calls computeBlockRecoveryWork().
*/ */
private class ReplicationMonitor implements Runnable { private class RedundancyMonitor implements Runnable {
@Override @Override
public void run() { public void run() {
while (namesystem.isRunning()) { while (namesystem.isRunning()) {
try { try {
// Process replication work only when active NN is out of safe mode. // Process recovery work only when active NN is out of safe mode.
if (isPopulatingReplQueues()) { if (isPopulatingReplQueues()) {
computeDatanodeWork(); computeDatanodeWork();
processPendingReconstructions(); processPendingReconstructions();
rescanPostponedMisreplicatedBlocks(); rescanPostponedMisreplicatedBlocks();
} }
Thread.sleep(replicationRecheckInterval); TimeUnit.MILLISECONDS.sleep(redundancyRecheckIntervalMs);
} catch (Throwable t) { } catch (Throwable t) {
if (!namesystem.isRunning()) { if (!namesystem.isRunning()) {
LOG.info("Stopping ReplicationMonitor."); LOG.info("Stopping RedundancyMonitor.");
if (!(t instanceof InterruptedException)) { if (!(t instanceof InterruptedException)) {
LOG.info("ReplicationMonitor received an exception" LOG.info("RedundancyMonitor received an exception"
+ " while shutting down.", t); + " while shutting down.", t);
} }
break; break;
} else if (!checkNSRunning && t instanceof InterruptedException) { } else if (!checkNSRunning && t instanceof InterruptedException) {
LOG.info("Stopping ReplicationMonitor for testing."); LOG.info("Stopping RedundancyMonitor for testing.");
break; break;
} }
LOG.error("ReplicationMonitor thread received Runtime exception. ", LOG.error("RedundancyMonitor thread received Runtime exception. ",
t); t);
terminate(1, t); terminate(1, t);
} }
@ -4692,6 +4696,14 @@ public class BlockManager implements BlockStatsMXBean {
} }
} }
/**
* @return redundancy thread.
*/
@VisibleForTesting
Daemon getRedundancyThread() {
return redundancyThread;
}
public BlockIdManager getBlockIdManager() { public BlockIdManager getBlockIdManager() {
return blockIdManager; return blockIdManager;
} }

View File

@ -83,11 +83,11 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
NetworkTopology clusterMap, NetworkTopology clusterMap,
Host2NodesMap host2datanodeMap) { Host2NodesMap host2datanodeMap) {
this.considerLoad = conf.getBoolean( this.considerLoad = conf.getBoolean(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_DEFAULT); DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_DEFAULT);
this.considerLoadFactor = conf.getDouble( this.considerLoadFactor = conf.getDouble(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_FACTOR, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_FACTOR_DEFAULT); DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR_DEFAULT);
this.stats = stats; this.stats = stats;
this.clusterMap = clusterMap; this.clusterMap = clusterMap;
this.host2datanodeMap = host2datanodeMap; this.host2datanodeMap = host2datanodeMap;

View File

@ -610,8 +610,8 @@ public class Mover {
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT,
TimeUnit.SECONDS) * 2000 + TimeUnit.SECONDS) * 2000 +
conf.getTimeDuration( conf.getTimeDuration(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT, DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
TimeUnit.SECONDS) * 1000; TimeUnit.SECONDS) * 1000;
AtomicInteger retryCount = new AtomicInteger(0); AtomicInteger retryCount = new AtomicInteger(0);
LOG.info("namenodes = " + namenodes); LOG.info("namenodes = " + namenodes);

View File

@ -469,11 +469,11 @@ public class BackupNode extends NameNode {
* (not run or not pass any control commands to DataNodes) * (not run or not pass any control commands to DataNodes)
* on BackupNode: * on BackupNode:
* {@link LeaseManager.Monitor} protected by SafeMode. * {@link LeaseManager.Monitor} protected by SafeMode.
* {@link BlockManager.ReplicationMonitor} protected by SafeMode. * {@link BlockManager.RedundancyMonitor} protected by SafeMode.
* {@link HeartbeatManager.Monitor} protected by SafeMode. * {@link HeartbeatManager.Monitor} protected by SafeMode.
* {@link DecommissionManager.Monitor} need to prohibit refreshNodes(). * {@link DecommissionManager.Monitor} need to prohibit refreshNodes().
* {@link PendingReconstructionBlocks.PendingReconstructionMonitor} * {@link PendingReconstructionBlocks.PendingReconstructionMonitor}
* harmless, because ReplicationMonitor is muted. * harmless, because RedundancyMonitor is muted.
*/ */
@Override @Override
public void startActiveServices() throws IOException { public void startActiveServices() throws IOException {

View File

@ -296,14 +296,14 @@
</property> </property>
<property> <property>
<name>dfs.namenode.replication.considerLoad</name> <name>dfs.namenode.redundancy.considerLoad</name>
<value>true</value> <value>true</value>
<description>Decide if chooseTarget considers the target's load or not <description>Decide if chooseTarget considers the target's load or not
</description> </description>
</property> </property>
<property> <property>
<name>dfs.namenode.replication.considerLoad.factor</name> <name>dfs.namenode.redundancy.considerLoad.factor</name>
<value>2.0</value> <value>2.0</value>
<description>The factor by which a node's load can exceed the average <description>The factor by which a node's load can exceed the average
before being rejected for writes, only if considerLoad is true. before being rejected for writes, only if considerLoad is true.
@ -980,10 +980,10 @@
</property> </property>
<property> <property>
<name>dfs.namenode.replication.interval</name> <name>dfs.namenode.redundancy.interval.seconds</name>
<value>3s</value> <value>3s</value>
<description>The periodicity in seconds with which the namenode computes <description>The periodicity in seconds with which the namenode computes
replication work for datanodes. Support multiple time unit suffix(case insensitive), low redundancy work for datanodes. Support multiple time unit suffix(case insensitive),
as described in dfs.heartbeat.interval. as described in dfs.heartbeat.interval.
</description> </description>
</property> </property>

View File

@ -90,14 +90,14 @@ public class AdminStatesBaseTest {
} }
// Setup conf // Setup conf
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
200); 200);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL);
conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
BLOCKREPORT_INTERVAL_MSEC); BLOCKREPORT_INTERVAL_MSEC);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
NAMENODE_REPLICATION_INTERVAL); NAMENODE_REPLICATION_INTERVAL);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);

View File

@ -58,7 +58,7 @@ public class TestBlockStoragePolicy {
static { static {
conf = new HdfsConfiguration(); conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
POLICY_SUITE = BlockStoragePolicySuite.createDefaultSuite(); POLICY_SUITE = BlockStoragePolicySuite.createDefaultSuite();
DEFAULT_STORAGE_POLICY = POLICY_SUITE.getDefaultPolicy(); DEFAULT_STORAGE_POLICY = POLICY_SUITE.getDefaultPolicy();
} }

View File

@ -67,7 +67,7 @@ public class TestDFSStripedOutputStream {
int numDNs = dataBlocks + parityBlocks + 2; int numDNs = dataBlocks + parityBlocks + 2;
conf = new Configuration(); conf = new Configuration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
if (ErasureCodeNative.isNativeCodeLoaded()) { if (ErasureCodeNative.isNativeCodeLoaded()) {

View File

@ -214,7 +214,7 @@ public class TestDFSStripedOutputStreamWithFailure {
private HdfsConfiguration newHdfsConfiguration() { private HdfsConfiguration newHdfsConfiguration() {
final HdfsConfiguration conf = new HdfsConfiguration(); final HdfsConfiguration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);

View File

@ -111,26 +111,26 @@ public class TestDecommissionWithStriped {
writeConfigFile(excludeFile, null); writeConfigFile(excludeFile, null);
// Setup conf // Setup conf
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath()); conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath()); conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
2000); 2000);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
BLOCKREPORT_INTERVAL_MSEC); BLOCKREPORT_INTERVAL_MSEC);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, conf.setInt(DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY,
4); 4);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
NAMENODE_REPLICATION_INTERVAL); NAMENODE_REPLICATION_INTERVAL);
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY, DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY,
cellSize - 1); cellSize - 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
numDNs = dataBlocks + parityBlocks + 2; numDNs = dataBlocks + parityBlocks + 2;

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hdfs; package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.junit.Test; import org.junit.Test;
@ -33,8 +34,23 @@ public class TestDeprecatedKeys {
String scriptFile = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY); String scriptFile = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY);
assertTrue(scriptFile.equals("xyz")) ; assertTrue(scriptFile.equals("xyz")) ;
conf.setInt("dfs.replication.interval", 1); conf.setInt("dfs.replication.interval", 1);
String alpha = DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY; int redundancyInterval = conf
int repInterval = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 3) ; .getInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 3);
assertTrue(repInterval == 1) ; assertTrue(redundancyInterval == 1);
int repInterval = conf.getInt("dfs.replication.interval", 3);
assertTrue(repInterval == 1);
repInterval = conf.getInt("dfs.namenode.replication.interval", 3);
assertTrue(repInterval == 1);
conf.setBoolean("dfs.replication.considerLoad", false);
assertFalse(conf.getBoolean(
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY, true));
assertFalse(conf.getBoolean("dfs.replication.considerLoad", true));
conf.setDouble("dfs.namenode.replication.considerLoad.factor", 5.0);
assertTrue(5.0 == conf.getDouble(
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR, 2.0));
assertTrue(5.0 == conf
.getDouble("dfs.namenode.replication.considerLoad.factor", 2.0));
} }
} }

View File

@ -329,7 +329,7 @@ public class TestEncryptedTransfer {
// client only retries establishing pipeline with the 4th node. // client only retries establishing pipeline with the 4th node.
int numDataNodes = 4; int numDataNodes = 4;
// do not consider load factor when selecting a data node // do not consider load factor when selecting a data node
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
setEncryptionConfigKeys(); setEncryptionConfigKeys();

View File

@ -86,7 +86,7 @@ public class TestFileAppend4 {
// handle under-replicated blocks quickly (for replication asserts) // handle under-replicated blocks quickly (for replication asserts)
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 5); DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 5);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
// handle failures in the DFSClient pipeline quickly // handle failures in the DFSClient pipeline quickly
// (for cluster.shutdown(); fs.close() idiom) // (for cluster.shutdown(); fs.close() idiom)

View File

@ -75,7 +75,7 @@ public class TestFileChecksum {
int numDNs = dataBlocks + parityBlocks + 2; int numDNs = dataBlocks + parityBlocks + 2;
conf = new Configuration(); conf = new Configuration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();

View File

@ -85,7 +85,7 @@ public class TestLeaseRecoveryStriped {
conf = new HdfsConfiguration(); conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L); conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);

View File

@ -59,7 +59,8 @@ public class TestMissingBlocksAlert {
try { try {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
//minimize test delay //minimize test delay
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
0);
conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
int fileLen = 10*1024; int fileLen = 10*1024;
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, fileLen/2); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, fileLen/2);

View File

@ -99,7 +99,8 @@ public class TestReadStripedFileWithDecoding {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY, 0);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
fs = cluster.getFileSystem(); fs = cluster.getFileSystem();

View File

@ -95,8 +95,8 @@ public class TestReconstructStripedFile {
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY, DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY,
cellSize - 1); cellSize - 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
if (ErasureCodeNative.isNativeCodeLoaded()) { if (ErasureCodeNative.isNativeCodeLoaded()) {
conf.set( conf.set(

View File

@ -118,7 +118,8 @@ public class TestReplaceDatanodeOnFailure {
public void testReplaceDatanodeOnFailure() throws Exception { public void testReplaceDatanodeOnFailure() throws Exception {
final Configuration conf = new HdfsConfiguration(); final Configuration conf = new HdfsConfiguration();
// do not consider load factor when selecting a data node // do not consider load factor when selecting a data node
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false);
//always replace a datanode //always replace a datanode
ReplaceDatanodeOnFailure.write(Policy.ALWAYS, true, conf); ReplaceDatanodeOnFailure.write(Policy.ALWAYS, true, conf);

View File

@ -228,7 +228,8 @@ public class TestReplication {
*/ */
public void runReplication(boolean simulated) throws IOException { public void runReplication(boolean simulated) throws IOException {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false);
if (simulated) { if (simulated) {
SimulatedFSDataset.setFactory(conf); SimulatedFSDataset.setFactory(conf);
} }

View File

@ -75,7 +75,7 @@ public class TestWriteReadStripedFile {
@Before @Before
public void setup() throws IOException { public void setup() throws IOException {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
fs = cluster.getFileSystem(); fs = cluster.getFileSystem();

View File

@ -179,7 +179,8 @@ public class TestBalancer {
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE); conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500); conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1L);
SimulatedFSDataset.setFactory(conf); SimulatedFSDataset.setFactory(conf);
conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L); conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
@ -212,10 +213,12 @@ public class TestBalancer {
void initConfWithStripe(Configuration conf) { void initConfWithStripe(Configuration conf) {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultBlockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultBlockSize);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
SimulatedFSDataset.setFactory(conf); SimulatedFSDataset.setFactory(conf);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1L);
conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L); conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
} }
@ -922,11 +925,13 @@ public class TestBalancer {
private static int runBalancer(Collection<URI> namenodes, private static int runBalancer(Collection<URI> namenodes,
final BalancerParameters p, final BalancerParameters p,
Configuration conf) throws IOException, InterruptedException { Configuration conf) throws IOException, InterruptedException {
final long sleeptime = final long sleeptime = conf.getLong(
conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 + DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000
conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, + conf.getLong(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000; DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT)
* 1000;
LOG.info("namenodes = " + namenodes); LOG.info("namenodes = " + namenodes);
LOG.info("parameters = " + p); LOG.info("parameters = " + p);
LOG.info("Print stack trace", new Throwable()); LOG.info("Print stack trace", new Throwable());
@ -1603,7 +1608,7 @@ public class TestBalancer {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500); conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
@ -1670,7 +1675,8 @@ public class TestBalancer {
int blockSize = 5 * 1024 * 1024 ; int blockSize = 5 * 1024 * 1024 ;
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1L);
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);

View File

@ -105,25 +105,24 @@ public class BlockManagerTestUtil {
} }
/** /**
* @return replication monitor thread instance from block manager. * @return redundancy monitor thread instance from block manager.
*/ */
public static Daemon getReplicationThread(final BlockManager blockManager) public static Daemon getRedundancyThread(final BlockManager blockManager) {
{ return blockManager.getRedundancyThread();
return blockManager.replicationThread;
} }
/** /**
* Stop the replication monitor thread * Stop the redundancy monitor thread.
*/ */
public static void stopReplicationThread(final BlockManager blockManager) public static void stopRedundancyThread(final BlockManager blockManager)
throws IOException { throws IOException {
blockManager.enableRMTerminationForTesting(); blockManager.enableRMTerminationForTesting();
blockManager.replicationThread.interrupt(); blockManager.getRedundancyThread().interrupt();
try { try {
blockManager.replicationThread.join(); blockManager.getRedundancyThread().join();
} catch(InterruptedException ie) { } catch (InterruptedException ie) {
throw new IOException( throw new IOException(
"Interrupted while trying to stop ReplicationMonitor"); "Interrupted while trying to stop RedundancyMonitor");
} }
} }

View File

@ -62,9 +62,9 @@ public class TestBlocksWithNotEnoughRacks {
// commands quickly (as replies to heartbeats) // commands quickly (as replies to heartbeats)
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
// Have the NN ReplicationMonitor compute the replication and // Have the NN RedundancyMonitor compute the reconstruction and
// invalidation commands to send DNs every second. // invalidation commands to send DNs every second.
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
// Have the NN check for pending replications every second so it // Have the NN check for pending replications every second so it
// quickly schedules additional replicas as they are identified. // quickly schedules additional replicas as they are identified.

View File

@ -56,7 +56,7 @@ public class TestNodeCount {
60); 60);
// reduce intervals to make test execution time shorter // reduce intervals to make test execution time shorter
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
// start a mini dfs cluster of 2 nodes // start a mini dfs cluster of 2 nodes

View File

@ -65,7 +65,7 @@ public class TestPendingInvalidateBlock {
// set the block report interval to 2s // set the block report interval to 2s
conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 2000); conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 2000);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
// disable the RPC timeout for debug // disable the RPC timeout for debug
conf.setLong(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY, 0); conf.setLong(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY, 0);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION) cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)

View File

@ -374,7 +374,7 @@ public class TestPendingReconstruction {
CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024); CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFS_REPLICATION_INTERVAL); DFS_REPLICATION_INTERVAL);
CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFS_REPLICATION_INTERVAL); DFS_REPLICATION_INTERVAL);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes( MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(
DATANODE_COUNT).build(); DATANODE_COUNT).build();

View File

@ -96,8 +96,8 @@ public class TestReconstructStripedBlocksWithRackAwareness {
@BeforeClass @BeforeClass
public static void setup() throws Exception { public static void setup() throws Exception {
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
} }
@ -139,7 +139,7 @@ public class TestReconstructStripedBlocksWithRackAwareness {
* *
* In this test, we first need to create a scenario that a striped block has * In this test, we first need to create a scenario that a striped block has
* all the internal blocks but distributed in <6 racks. Then we check if the * all the internal blocks but distributed in <6 racks. Then we check if the
* replication monitor can correctly schedule the reconstruction work for it. * redundancy monitor can correctly schedule the reconstruction work for it.
*/ */
@Test @Test
public void testReconstructForNotEnoughRacks() throws Exception { public void testReconstructForNotEnoughRacks() throws Exception {
@ -194,7 +194,7 @@ public class TestReconstructStripedBlocksWithRackAwareness {
fsn.writeUnlock(); fsn.writeUnlock();
} }
// check if replication monitor correctly schedule the replication work // check if redundancy monitor correctly schedule the reconstruction work.
boolean scheduled = false; boolean scheduled = false;
for (int i = 0; i < 5; i++) { // retry 5 times for (int i = 0; i < 5; i++) { // retry 5 times
for (DatanodeStorageInfo storage : blockInfo.storages) { for (DatanodeStorageInfo storage : blockInfo.storages) {

View File

@ -836,7 +836,7 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
int DFS_NAMENODE_REPLICATION_INTERVAL = 1000; int DFS_NAMENODE_REPLICATION_INTERVAL = 1000;
int HIGH_PRIORITY = 0; int HIGH_PRIORITY = 0;
Configuration conf = new Configuration(); Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2) MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
.format(true).build(); .format(true).build();
try { try {

View File

@ -53,7 +53,7 @@ public class TestReplicationPolicyConsiderLoad
@Override @Override
DatanodeDescriptor[] getDatanodeDescriptors(Configuration conf) { DatanodeDescriptor[] getDatanodeDescriptors(Configuration conf) {
conf.setDouble(DFSConfigKeys conf.setDouble(DFSConfigKeys
.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_FACTOR, 1.2); .DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_FACTOR, 1.2);
final String[] racks = { final String[] racks = {
"/rack1", "/rack1",
"/rack1", "/rack1",

View File

@ -78,7 +78,7 @@ public class TestDataNodeErasureCodingMetrics {
conf = new Configuration(); conf = new Configuration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
cluster.waitActive(); cluster.waitActive();
cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);

View File

@ -104,8 +104,8 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
Thread.sleep(30000L); Thread.sleep(30000L);
assertThat(cluster.getNamesystem().getNumDeadDataNodes(), is(1)); assertThat(cluster.getNamesystem().getNumDeadDataNodes(), is(1));
// Next, wait for the replication monitor to mark the file as corrupt // Next, wait for the redundancy monitor to mark the file as corrupt.
Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000); Thread.sleep(2 * DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT * 1000);
// Wait for the LazyPersistFileScrubber to run // Wait for the LazyPersistFileScrubber to run
Thread.sleep(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC * 1000); Thread.sleep(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC * 1000);
@ -137,8 +137,8 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
cluster.shutdownDataNodes(); cluster.shutdownDataNodes();
Thread.sleep(30000L); Thread.sleep(30000L);
// Next, wait for the replication monitor to mark the file as corrupt // Next, wait for the redundancy monitor to mark the file as corrupt.
Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000); Thread.sleep(2 * DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT * 1000);
// Wait for the LazyPersistFileScrubber to run // Wait for the LazyPersistFileScrubber to run
Thread.sleep(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC * 1000); Thread.sleep(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC * 1000);
@ -164,8 +164,8 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
cluster.restartNameNodes(); cluster.restartNameNodes();
// wait for the replication monitor to mark the file as corrupt // wait for the redundancy monitor to mark the file as corrupt.
Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000); Thread.sleep(2 * DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT * 1000);
Long corruptBlkCount = (long) Iterators.size(cluster.getNameNode() Long corruptBlkCount = (long) Iterators.size(cluster.getNameNode()
.getNamesystem().getBlockManager().getCorruptReplicaBlockIterator()); .getNamesystem().getBlockManager().getCorruptReplicaBlockIterator());

View File

@ -105,7 +105,8 @@ public class TestMover {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE); conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1L);
conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L); conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
} }
@ -482,8 +483,10 @@ public class TestMover {
void initConfWithStripe(Configuration conf) { void initConfWithStripe(Configuration conf) {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultBlockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultBlockSize);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); 1L);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false);
} }
@Test(timeout = 300000) @Test(timeout = 300000)

View File

@ -93,8 +93,8 @@ public class TestStorageMover {
static { static {
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); DEFAULT_CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); DEFAULT_CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, DEFAULT_CONF.setLong(
2L); DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 2L);
DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L); DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L);
DEFAULT_POLICIES = BlockStoragePolicySuite.createDefaultSuite(); DEFAULT_POLICIES = BlockStoragePolicySuite.createDefaultSuite();

View File

@ -1237,7 +1237,7 @@ public class NNThroughputBenchmark implements Tool {
} // end BlockReportStats } // end BlockReportStats
/** /**
* Measures how fast replication monitor can compute data-node work. * Measures how fast redundancy monitor can compute data-node work.
* *
* It runs only one thread until no more work can be scheduled. * It runs only one thread until no more work can be scheduled.
*/ */
@ -1265,7 +1265,7 @@ public class NNThroughputBenchmark implements Tool {
parseArguments(args); parseArguments(args);
// number of operations is 4 times the number of decommissioned // number of operations is 4 times the number of decommissioned
// blocks divided by the number of needed replications scanned // blocks divided by the number of needed replications scanned
// by the replication monitor in one iteration // by the redundancy monitor in one iteration
numOpsRequired = (totalBlocks*replication*nodesToDecommission*2) numOpsRequired = (totalBlocks*replication*nodesToDecommission*2)
/ (numDatanodes*numDatanodes); / (numDatanodes*numDatanodes);
@ -1314,8 +1314,8 @@ public class NNThroughputBenchmark implements Tool {
// start data-nodes; create a bunch of files; generate block reports. // start data-nodes; create a bunch of files; generate block reports.
blockReportObject.generateInputs(ignore); blockReportObject.generateInputs(ignore);
// stop replication monitor // stop redundancy monitor thread.
BlockManagerTestUtil.stopReplicationThread(namesystem.getBlockManager()); BlockManagerTestUtil.stopRedundancyThread(namesystem.getBlockManager());
// report blocks once // report blocks once
int nrDatanodes = blockReportObject.getNumDatanodes(); int nrDatanodes = blockReportObject.getNumDatanodes();

View File

@ -74,7 +74,7 @@ public class TestAddOverReplicatedStripedBlocks {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
// disable block recovery // disable block recovery
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
SimulatedFSDataset.setFactory(conf); SimulatedFSDataset.setFactory(conf);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();

View File

@ -78,7 +78,7 @@ public class TestDecommissioningStatus {
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
conf = new HdfsConfiguration(); conf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
// Set up the hosts/exclude files. // Set up the hosts/exclude files.
@ -89,7 +89,8 @@ public class TestDecommissioningStatus {
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt( conf.setInt(
DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 4); DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 4);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1000);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
conf.setLong(DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY, 1); conf.setLong(DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY, 1);

View File

@ -156,7 +156,7 @@ public class TestFSEditLogLoader {
// start a cluster // start a cluster
Configuration conf = getConf(); Configuration conf = getConf();
// Replicate and heartbeat fast to shave a few seconds off test // Replicate and heartbeat fast to shave a few seconds off test
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
MiniDFSCluster cluster = null; MiniDFSCluster cluster = null;

View File

@ -76,9 +76,9 @@ public class TestHostsFiles {
// commands quickly (as replies to heartbeats) // commands quickly (as replies to heartbeats)
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
// Have the NN ReplicationMonitor compute the replication and // Have the NN RedundancyMonitor compute the low redundant blocks and
// invalidation commands to send DNs every second. // invalidation commands to send DNs every second.
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
// Have the NN check for pending replications every second so it // Have the NN check for pending replications every second so it
// quickly schedules additional replicas as they are identified. // quickly schedules additional replicas as they are identified.

View File

@ -64,8 +64,9 @@ public class TestMetaSave {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
// High value of replication interval // High value of replication interval
// so that blocks remain under-replicated // so that blocks remain less redundant
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1000);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1L);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY, 5L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY, 5L);

View File

@ -192,7 +192,8 @@ public class TestReconstructStripedBlocks {
public void test2RecoveryTasksForSameBlockGroup() throws Exception { public void test2RecoveryTasksForSameBlockGroup() throws Exception {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1000); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1000);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1000);
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(groupSize + 2) cluster = new MiniDFSCluster.Builder(conf).numDataNodes(groupSize + 2)
.build(); .build();
@ -255,8 +256,8 @@ public class TestReconstructStripedBlocks {
@Test @Test
public void testCountLiveReplicas() throws Exception { public void testCountLiveReplicas() throws Exception {
final HdfsConfiguration conf = new HdfsConfiguration(); final HdfsConfiguration conf = new HdfsConfiguration();
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(groupSize + 2) cluster = new MiniDFSCluster.Builder(conf).numDataNodes(groupSize + 2)
.build(); .build();
@ -299,7 +300,7 @@ public class TestReconstructStripedBlocks {
FSNamesystem fsn = cluster.getNamesystem(); FSNamesystem fsn = cluster.getNamesystem();
BlockManager bm = fsn.getBlockManager(); BlockManager bm = fsn.getBlockManager();
Thread.sleep(3000); // wait 3 running cycles of replication monitor Thread.sleep(3000); // wait 3 running cycles of redundancy monitor
for (DataNode dn : cluster.getDataNodes()) { for (DataNode dn : cluster.getDataNodes()) {
DataNodeTestUtils.triggerHeartbeat(dn); DataNodeTestUtils.triggerHeartbeat(dn);
} }

View File

@ -307,8 +307,9 @@ public class TestStripedINodeFile {
int defaultStripedBlockSize = testECPolicy.getCellSize() * 4; int defaultStripedBlockSize = testECPolicy.getCellSize() * 4;
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultStripedBlockSize); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultStripedBlockSize);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, 1L);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
false); false);
// start 10 datanodes // start 10 datanodes

View File

@ -87,9 +87,10 @@ public class TestDNFencing {
public void setupCluster() throws Exception { public void setupCluster() throws Exception {
conf = new Configuration(); conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, SMALL_BLOCK); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, SMALL_BLOCK);
// Bump up replication interval so that we only run replication // Bump up redundancy interval so that we only run low redundancy
// checks explicitly. // checks explicitly.
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 600); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
600);
// Increase max streams so that we re-replicate quickly. // Increase max streams so that we re-replicate quickly.
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 1000); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 1000);
// See RandomDeleterPolicy javadoc. // See RandomDeleterPolicy javadoc.
@ -167,7 +168,7 @@ public class TestDNFencing {
// The blocks should no longer be postponed. // The blocks should no longer be postponed.
assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks()); assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());
// Wait for NN2 to enact its deletions (replication monitor has to run, etc) // Wait for NN2 to enact its deletions (redundancy monitor has to run, etc)
BlockManagerTestUtil.computeInvalidationWork( BlockManagerTestUtil.computeInvalidationWork(
nn2.getNamesystem().getBlockManager()); nn2.getNamesystem().getBlockManager());
cluster.triggerHeartbeats(); cluster.triggerHeartbeats();
@ -258,7 +259,7 @@ public class TestDNFencing {
// The block should no longer be postponed. // The block should no longer be postponed.
assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks()); assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());
// Wait for NN2 to enact its deletions (replication monitor has to run, etc) // Wait for NN2 to enact its deletions (redundancy monitor has to run, etc)
BlockManagerTestUtil.computeInvalidationWork( BlockManagerTestUtil.computeInvalidationWork(
nn2.getNamesystem().getBlockManager()); nn2.getNamesystem().getBlockManager());
@ -358,7 +359,7 @@ public class TestDNFencing {
// The block should no longer be postponed. // The block should no longer be postponed.
assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks()); assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());
// Wait for NN2 to enact its deletions (replication monitor has to run, etc) // Wait for NN2 to enact its deletions (redundancy monitor has to run, etc)
BlockManagerTestUtil.computeInvalidationWork( BlockManagerTestUtil.computeInvalidationWork(
nn2.getNamesystem().getBlockManager()); nn2.getNamesystem().getBlockManager());

View File

@ -110,7 +110,7 @@ public class TestDNFencingWithReplication {
harness.conf.setInt( harness.conf.setInt(
DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1); DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1);
harness.conf.setInt( harness.conf.setInt(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
final MiniDFSCluster cluster = harness.startCluster(); final MiniDFSCluster cluster = harness.startCluster();
try { try {

View File

@ -142,8 +142,9 @@ public class TestPipelinesFailover {
MethodToTestIdempotence methodToTest) throws Exception { MethodToTestIdempotence methodToTest) throws Exception {
Configuration conf = new Configuration(); Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
// Don't check replication periodically. // Don't check low redundancy periodically.
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000); conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
1000);
FSDataOutputStream stm = null; FSDataOutputStream stm = null;
MiniDFSCluster cluster = newMiniCluster(conf, 3); MiniDFSCluster cluster = newMiniCluster(conf, 3);

View File

@ -45,7 +45,7 @@ public class TestNNMetricFilesInGetListingOps {
CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100); CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1); CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
} }
private MiniDFSCluster cluster; private MiniDFSCluster cluster;

View File

@ -78,7 +78,7 @@ import org.junit.Test;
*/ */
public class TestNameNodeMetrics { public class TestNameNodeMetrics {
private static final Configuration CONF = new HdfsConfiguration(); private static final Configuration CONF = new HdfsConfiguration();
private static final int DFS_REPLICATION_INTERVAL = 1; private static final int DFS_REDUNDANCY_INTERVAL = 1;
private static final Path TEST_ROOT_DIR_PATH = private static final Path TEST_ROOT_DIR_PATH =
new Path("/testNameNodeMetrics"); new Path("/testNameNodeMetrics");
private static final String NN_METRICS = "NameNodeActivity"; private static final String NN_METRICS = "NameNodeActivity";
@ -96,9 +96,9 @@ public class TestNameNodeMetrics {
CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100); CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1); CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFS_REPLICATION_INTERVAL); DFS_REDUNDANCY_INTERVAL);
CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
DFS_REPLICATION_INTERVAL); DFS_REDUNDANCY_INTERVAL);
CONF.set(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY, CONF.set(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
"" + PERCENTILES_INTERVAL); "" + PERCENTILES_INTERVAL);
// Enable stale DataNodes checking // Enable stale DataNodes checking
@ -333,7 +333,7 @@ public class TestNameNodeMetrics {
private void waitForDeletion() throws InterruptedException { private void waitForDeletion() throws InterruptedException {
// Wait for more than DATANODE_COUNT replication intervals to ensure all // Wait for more than DATANODE_COUNT replication intervals to ensure all
// the blocks pending deletion are sent for deletion to the datanodes. // the blocks pending deletion are sent for deletion to the datanodes.
Thread.sleep(DFS_REPLICATION_INTERVAL * (DATANODE_COUNT + 1) * 1000); Thread.sleep(DFS_REDUNDANCY_INTERVAL * (DATANODE_COUNT + 1) * 1000);
} }
/** /**
@ -364,7 +364,7 @@ public class TestNameNodeMetrics {
rb = getMetrics(source); rb = getMetrics(source);
gauge = MetricsAsserts.getLongGauge(name, rb); gauge = MetricsAsserts.getLongGauge(name, rb);
while (gauge != expected && (--retries > 0)) { while (gauge != expected && (--retries > 0)) {
Thread.sleep(DFS_REPLICATION_INTERVAL * 500); Thread.sleep(DFS_REDUNDANCY_INTERVAL * 500);
rb = getMetrics(source); rb = getMetrics(source);
gauge = MetricsAsserts.getLongGauge(name, rb); gauge = MetricsAsserts.getLongGauge(name, rb);
} }