HDFS-15659. MiniDFSCluster dfs.namenode.redundancy.considerLoad default to false (#2443). Contributed by Ahmed Hussein.
This commit is contained in:
parent
e0ec1849d7
commit
9731dfbf0d
|
@ -21,6 +21,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME
|
||||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY;
|
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY;
|
||||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_DEFAULT;
|
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_DEFAULT;
|
||||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
|
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
|
||||||
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY;
|
||||||
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
|
||||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SCANNER_VOLUME_JOIN_TIMEOUT_MSEC_KEY;
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SCANNER_VOLUME_JOIN_TIMEOUT_MSEC_KEY;
|
||||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
|
||||||
|
@ -169,6 +170,10 @@ public class MiniDFSCluster implements AutoCloseable {
|
||||||
// Changing this default may break some tests that assume it is 2.
|
// Changing this default may break some tests that assume it is 2.
|
||||||
private static final int DEFAULT_STORAGES_PER_DATANODE = 2;
|
private static final int DEFAULT_STORAGES_PER_DATANODE = 2;
|
||||||
|
|
||||||
|
// do not consider load factor when selecting a datanode.
|
||||||
|
private static final boolean DEFAULT_DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD =
|
||||||
|
false;
|
||||||
|
|
||||||
static { DefaultMetricsSystem.setMiniClusterMode(true); }
|
static { DefaultMetricsSystem.setMiniClusterMode(true); }
|
||||||
|
|
||||||
public int getStoragesPerDatanode() {
|
public int getStoragesPerDatanode() {
|
||||||
|
@ -478,6 +483,19 @@ public class MiniDFSCluster implements AutoCloseable {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* set the value of DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY in the config
|
||||||
|
* file.
|
||||||
|
*
|
||||||
|
* @param val passed to the flag. This allows overriding the default value
|
||||||
|
* {@link #DEFAULT_DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD}.
|
||||||
|
* @return the builder object.
|
||||||
|
*/
|
||||||
|
public Builder setNNRedundancyConsiderLoad(final boolean val) {
|
||||||
|
conf.setBoolean(DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY, val);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Construct the actual MiniDFSCluster
|
* Construct the actual MiniDFSCluster
|
||||||
*/
|
*/
|
||||||
|
@ -494,6 +512,10 @@ public class MiniDFSCluster implements AutoCloseable {
|
||||||
DEFAULT_SCANNER_VOLUME_JOIN_TIMEOUT_MSEC);
|
DEFAULT_SCANNER_VOLUME_JOIN_TIMEOUT_MSEC);
|
||||||
conf.setLong(DFS_BLOCK_SCANNER_VOLUME_JOIN_TIMEOUT_MSEC_KEY,
|
conf.setLong(DFS_BLOCK_SCANNER_VOLUME_JOIN_TIMEOUT_MSEC_KEY,
|
||||||
defaultScannerVolumeTimeOut);
|
defaultScannerVolumeTimeOut);
|
||||||
|
// default is false. do not consider load factor when selecting a
|
||||||
|
// datanode.
|
||||||
|
conf.setBoolean(DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
||||||
|
DEFAULT_DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD);
|
||||||
this.storagesPerDatanode =
|
this.storagesPerDatanode =
|
||||||
FsDatasetTestUtils.Factory.getFactory(conf).getDefaultNumOfDataDirs();
|
FsDatasetTestUtils.Factory.getFactory(conf).getDefaultNumOfDataDirs();
|
||||||
}
|
}
|
||||||
|
|
|
@ -76,8 +76,6 @@ abstract public class ReadStripedFileWithDecodingHelper {
|
||||||
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,
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
|
||||||
0);
|
0);
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
MiniDFSCluster myCluster = new MiniDFSCluster.Builder(conf)
|
MiniDFSCluster myCluster = new MiniDFSCluster.Builder(conf)
|
||||||
.numDataNodes(NUM_DATANODES)
|
.numDataNodes(NUM_DATANODES)
|
||||||
.build();
|
.build();
|
||||||
|
|
|
@ -846,10 +846,8 @@ public class TestDFSClientRetries {
|
||||||
public void testGetFileChecksum() throws Exception {
|
public void testGetFileChecksum() throws Exception {
|
||||||
final String f = "/testGetFileChecksum";
|
final String f = "/testGetFileChecksum";
|
||||||
final Path p = new Path(f);
|
final Path p = new Path(f);
|
||||||
// HDFS-15461: the number of datanode is higher than the number of replicas.
|
|
||||||
// That way when a DN fails, the pipeline can recover.
|
|
||||||
final int numReplicas = 3;
|
final int numReplicas = 3;
|
||||||
final int numDatanodes = numReplicas + 1;
|
final int numDatanodes = numReplicas;
|
||||||
final MiniDFSCluster cluster =
|
final MiniDFSCluster cluster =
|
||||||
new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).build();
|
new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).build();
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -118,8 +118,6 @@ public class TestDecommissionWithStriped {
|
||||||
writeConfigFile(excludeFile, null);
|
writeConfigFile(excludeFile, null);
|
||||||
|
|
||||||
// Setup conf
|
// Setup conf
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
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,
|
||||||
|
|
|
@ -381,9 +381,7 @@ public class TestEncryptedTransfer {
|
||||||
// use 4 datanodes to make sure that after 1 data node is stopped,
|
// use 4 datanodes to make sure that after 1 data node is stopped,
|
||||||
// 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
|
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
setEncryptionConfigKeys();
|
setEncryptionConfigKeys();
|
||||||
|
|
||||||
cluster = new MiniDFSCluster.Builder(conf)
|
cluster = new MiniDFSCluster.Builder(conf)
|
||||||
|
|
|
@ -88,8 +88,6 @@ 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_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
|
||||||
conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
|
conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
|
||||||
customizeConf(conf);
|
customizeConf(conf);
|
||||||
|
|
|
@ -92,8 +92,6 @@ 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, 60000L);
|
conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 60000L);
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
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);
|
||||||
final int numDNs = dataBlocks + parityBlocks;
|
final int numDNs = dataBlocks + parityBlocks;
|
||||||
|
|
|
@ -140,8 +140,6 @@ public class TestReconstructStripedFile {
|
||||||
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_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
if (ErasureCodeNative.isNativeCodeLoaded()) {
|
if (ErasureCodeNative.isNativeCodeLoaded()) {
|
||||||
conf.set(
|
conf.set(
|
||||||
CodecUtil.IO_ERASURECODE_CODEC_RS_RAWCODERS_KEY,
|
CodecUtil.IO_ERASURECODE_CODEC_RS_RAWCODERS_KEY,
|
||||||
|
|
|
@ -339,8 +339,6 @@ 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_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
if (simulated) {
|
if (simulated) {
|
||||||
SimulatedFSDataset.setFactory(conf);
|
SimulatedFSDataset.setFactory(conf);
|
||||||
}
|
}
|
||||||
|
|
|
@ -75,8 +75,6 @@ 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_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
|
||||||
fs = cluster.getFileSystem();
|
fs = cluster.getFileSystem();
|
||||||
fs.enableErasureCodingPolicy(ecPolicy.getName());
|
fs.enableErasureCodingPolicy(ecPolicy.getName());
|
||||||
|
|
|
@ -277,8 +277,6 @@ public class TestReconstructStripedBlocks {
|
||||||
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_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(groupSize + 2)
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(groupSize + 2)
|
||||||
.build();
|
.build();
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
|
|
|
@ -393,8 +393,6 @@ public class TestStripedINodeFile {
|
||||||
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
||||||
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
|
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
|
||||||
1L);
|
1L);
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
|
|
||||||
// start 10 datanodes
|
// start 10 datanodes
|
||||||
int numOfDatanodes = 10;
|
int numOfDatanodes = 10;
|
||||||
|
|
|
@ -60,8 +60,6 @@ public class TestHAAppend {
|
||||||
// control the ingest of edits by the standby for this test.
|
// control the ingest of edits by the standby for this test.
|
||||||
conf.set(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, "5000");
|
conf.set(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, "5000");
|
||||||
conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, -1);
|
conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, -1);
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
|
|
||||||
false);
|
|
||||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
||||||
.nnTopology(MiniDFSNNTopology.simpleHATopology())
|
.nnTopology(MiniDFSNNTopology.simpleHATopology())
|
||||||
.numDataNodes(3).build();
|
.numDataNodes(3).build();
|
||||||
|
|
Loading…
Reference in New Issue