HDFS-8084. Move dfs.client.failover.* confs from DFSConfigKeys to HdfsClientConfigKeys.Failover and fix typos in the dfs.http.client.* configuration keys.

This commit is contained in:
Tsz-Wo Nicholas Sze 2015-04-10 19:38:43 -07:00
parent 36e4cd3be6
commit 60da0e49e7
17 changed files with 238 additions and 181 deletions

View File

@ -28,53 +28,64 @@ public interface HdfsClientConfigKeys {
"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
static final String PREFIX = "dfs.client.";
/** Client retry configuration properties */
public interface Retry {
static final String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
public static final String POLICY_ENABLED_KEY
= PREFIX + "policy.enabled";
public static final boolean POLICY_ENABLED_DEFAULT
= false;
public static final String POLICY_SPEC_KEY
= PREFIX + "policy.spec";
public static final String POLICY_SPEC_DEFAULT
= "10000,6,60000,10"; //t1,n1,t2,n2,...
String POLICY_ENABLED_KEY = PREFIX + "policy.enabled";
boolean POLICY_ENABLED_DEFAULT = false;
String POLICY_SPEC_KEY = PREFIX + "policy.spec";
String POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
public static final String TIMES_GET_LAST_BLOCK_LENGTH_KEY
= PREFIX + "times.get-last-block-length";
public static final int TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT
= 3;
public static final String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY
= PREFIX + "interval-ms.get-last-block-length";
public static final int INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT
= 4000;
String TIMES_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "times.get-last-block-length";
int TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT = 3;
String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "interval-ms.get-last-block-length";
int INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
public static final String MAX_ATTEMPTS_KEY
= PREFIX + "max.attempts";
public static final int MAX_ATTEMPTS_DEFAULT
= 10;
String MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
int MAX_ATTEMPTS_DEFAULT = 10;
public static final String WINDOW_BASE_KEY
= PREFIX + "window.base";
public static final int WINDOW_BASE_DEFAULT
= 3000;
String WINDOW_BASE_KEY = PREFIX + "window.base";
int WINDOW_BASE_DEFAULT = 3000;
}
// WebHDFS retry configuration policy
interface WebHdfsRetry {
String PREFIX = HdfsClientConfigKeys.PREFIX + "http.client.";
String RETRY_POLICY_ENABLED_KEY = PREFIX + "dfs.http.client.retry.policy.enabled";
/** Client failover configuration properties */
interface Failover {
String PREFIX = HdfsClientConfigKeys.PREFIX + "failover.";
String PROXY_PROVIDER_KEY_PREFIX = PREFIX + "proxy.provider";
String MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
int MAX_ATTEMPTS_DEFAULT = 15;
String SLEEPTIME_BASE_KEY = PREFIX + "sleep.base.millis";
int SLEEPTIME_BASE_DEFAULT = 500;
String SLEEPTIME_MAX_KEY = PREFIX + "sleep.max.millis";
int SLEEPTIME_MAX_DEFAULT = 15000;
String CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
int CONNECTION_RETRIES_DEFAULT = 0;
String CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = PREFIX + "connection.retries.on.timeouts";
int CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
}
/** HTTP client configuration properties */
interface HttpClient {
String PREFIX = "dfs.http.client.";
// retry
String RETRY_POLICY_ENABLED_KEY = PREFIX + "retry.policy.enabled";
boolean RETRY_POLICY_ENABLED_DEFAULT = false;
String RETRY_POLICY_SPEC_KEY = PREFIX + "dfs.http.client.retry.policy.spec";
String RETRY_POLICY_SPEC_KEY = PREFIX + "retry.policy.spec";
String RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
String FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "dfs.http.client.failover.max.attempts";
int FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;
String RETRY_MAX_ATTEMPTS_KEY = PREFIX + "dfs.http.client.retry.max.attempts";
String RETRY_MAX_ATTEMPTS_KEY = PREFIX + "retry.max.attempts";
int RETRY_MAX_ATTEMPTS_DEFAULT = 10;
String FAILOVER_SLEEPTIME_BASE_KEY = PREFIX + "dfs.http.client.failover.sleep.base.millis";
// failover
String FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "failover.max.attempts";
int FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;
String FAILOVER_SLEEPTIME_BASE_KEY = PREFIX + "failover.sleep.base.millis";
int FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
String FAILOVER_SLEEPTIME_MAX_KEY = PREFIX + "dfs.http.client.failover.sleep.max.millis";
String FAILOVER_SLEEPTIME_MAX_KEY = PREFIX + "failover.sleep.max.millis";
int FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
}
}
}

View File

@ -421,6 +421,10 @@ Release 2.8.0 - UNRELEASED
HDFS-8103. Move BlockTokenSecretManager.AccessMode into
BlockTokenIdentifier. (wheat9)
HDFS-8084. Move dfs.client.failover.* confs from DFSConfigKeys to
HdfsClientConfigKeys.Failover and fix typos in the dfs.http.client.*
configuration keys. (szetszwo)
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -603,40 +603,40 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
// WebHDFS retry policy
@Deprecated
public static final String DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_KEY =
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_KEY;
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_KEY;
@Deprecated
public static final boolean DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_DEFAULT =
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_DEFAULT;
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_DEFAULT;
@Deprecated
public static final String DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_KEY =
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_KEY;
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_KEY;
@Deprecated
public static final String DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_DEFAULT =
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_DEFAULT;
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_DEFAULT;
@Deprecated
public static final String DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY =
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_KEY;
HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_KEY;
@Deprecated
public static final int DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT =
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_DEFAULT;
HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_DEFAULT;
@Deprecated
public static final String DFS_HTTP_CLIENT_RETRY_MAX_ATTEMPTS_KEY =
HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_KEY;
HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_KEY;
@Deprecated
public static final int DFS_HTTP_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT =
HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_DEFAULT;
HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_DEFAULT;
@Deprecated
public static final String DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY =
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_KEY;
HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_KEY;
@Deprecated
public static final int DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT =
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_DEFAULT;
HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_DEFAULT;
@Deprecated
public static final String DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY =
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_KEY;
HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_KEY;
@Deprecated
public static final int DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT
= HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_DEFAULT;
= HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_DEFAULT;
// Handling unresolved DN topology mapping
public static final String DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_KEY =
@ -688,7 +688,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
// client retry confs are moved to HdfsConfigKeys.Client.Retry
// client retry confs are moved to HdfsClientConfigKeys.Retry
@Deprecated
public static final String DFS_CLIENT_RETRY_POLICY_ENABLED_KEY
= HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY;
@ -726,6 +726,40 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_CLIENT_RETRY_WINDOW_BASE_DEFAULT
= HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT;
// client failover confs are moved to HdfsClientConfigKeys.Failover
@Deprecated
public static final String DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX
= HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX;
@Deprecated
public static final String DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY
= HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY;
@Deprecated
public static final int DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT
= HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT;
@Deprecated
public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY
= HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY;
@Deprecated
public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT
= HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT;
@Deprecated
public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY
= HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY;
@Deprecated
public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT
= HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT;
@Deprecated
public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY
= HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY;
@Deprecated
public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT
= HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT;
@Deprecated
public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY
= HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY;
@Deprecated
public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT
= HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT;
@ -779,18 +813,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT = 10;
public static final String DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS = "dfs.client.file-block-storage-locations.timeout.millis";
public static final int DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT = 1000;
public static final String DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX = "dfs.client.failover.proxy.provider";
public static final String DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY = "dfs.client.failover.max.attempts";
public static final int DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;
public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY = "dfs.client.failover.sleep.base.millis";
public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY = "dfs.client.failover.sleep.max.millis";
public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY = "dfs.client.failover.connection.retries";
public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT = 0;
public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = "dfs.client.failover.connection.retries.on.timeouts";
public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
public static final String DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY = "dfs.client.datanode-restart.timeout";
public static final long DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT = 30;

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
@ -39,6 +38,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.NameNodeProxies.ProxyAndInfo;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@ -227,8 +227,8 @@ public static boolean isLogicalUri(
public static boolean isClientFailoverConfigured(
Configuration conf, URI nameNodeUri) {
String host = nameNodeUri.getHost();
String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
+ host;
String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+ "." + host;
return conf.get(configKey) != null;
}

View File

@ -17,14 +17,6 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationHandler;
@ -230,14 +222,14 @@ public static <T> ProxyAndInfo<T> createProxyWithLossyRetryHandler(
if (failoverProxyProvider != null) { // HA case
int delay = config.getInt(
DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
int maxCap = config.getInt(
DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
int maxFailoverAttempts = config.getInt(
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
int maxRetryAttempts = config.getInt(
HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
@ -469,9 +461,8 @@ public static <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass(
return null;
}
String host = nameNodeUri.getHost();
String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
+ host;
String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+ "." + host;
try {
@SuppressWarnings("unchecked")
Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>) conf

View File

@ -31,12 +31,6 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY;
@ -119,18 +113,28 @@ public DfsClientConf(Configuration conf) {
// The hdfsTimeout is currently the same as the ipc timeout
hdfsTimeout = Client.getTimeout(conf);
maxFailoverAttempts = conf.getInt(
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
maxRetryAttempts = conf.getInt(
HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
timeWindow = conf.getInt(
HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY,
HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT);
retryTimesForGetLastBlockLength = conf.getInt(
HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY,
HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
retryIntervalForGetLastBlockLength = conf.getInt(
HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_KEY,
HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
maxFailoverAttempts = conf.getInt(
HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
failoverSleepBaseMillis = conf.getInt(
DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
failoverSleepMaxMillis = conf.getInt(
DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
maxBlockAcquireFailures = conf.getInt(
DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
@ -179,9 +183,6 @@ public DfsClientConf(Configuration conf) {
DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
10 * defaultBlockSize);
timeWindow = conf.getInt(
HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY,
HdfsClientConfigKeys.Retry.WINDOW_BASE_DEFAULT);
numCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
numBlockWriteRetry = conf.getInt(DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
@ -204,13 +205,6 @@ public DfsClientConf(Configuration conf) {
fileBlockStorageLocationsTimeoutMs = conf.getInt(
DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS,
DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT);
retryTimesForGetLastBlockLength = conf.getInt(
HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY,
HdfsClientConfigKeys.Retry.TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
retryIntervalForGetLastBlockLength = conf.getInt(
HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_KEY,
HdfsClientConfigKeys.Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
datanodeRestartTimeout = conf.getLong(
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,

View File

@ -30,11 +30,10 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation;
@ -69,15 +68,15 @@ public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
this.conf = new Configuration(conf);
int maxRetries = this.conf.getInt(
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY,
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT);
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
this.conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
maxRetries);
int maxRetriesOnSocketTimeouts = this.conf.getInt(
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
this.conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
maxRetriesOnSocketTimeouts);

View File

@ -24,11 +24,10 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.io.retry.FailoverProxyProvider;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation;
@ -67,15 +66,15 @@ public IPFailoverProxyProvider(Configuration conf, URI uri,
this.conf = new Configuration(conf);
int maxRetries = this.conf.getInt(
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY,
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT);
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
this.conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
maxRetries);
int maxRetriesOnSocketTimeouts = this.conf.getInt(
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
this.conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
maxRetriesOnSocketTimeouts);

View File

@ -173,25 +173,25 @@ public synchronized void initialize(URI uri, Configuration conf
this.retryPolicy =
RetryUtils.getDefaultRetryPolicy(
conf,
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_KEY,
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_DEFAULT,
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_KEY,
HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_SPEC_DEFAULT,
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_KEY,
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_DEFAULT,
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_KEY,
HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_DEFAULT,
SafeModeException.class);
} else {
int maxFailoverAttempts = conf.getInt(
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_MAX_ATTEMPTS_DEFAULT);
HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.HttpClient.FAILOVER_MAX_ATTEMPTS_DEFAULT);
int maxRetryAttempts = conf.getInt(
HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.WebHdfsRetry.RETRY_MAX_ATTEMPTS_DEFAULT);
HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_KEY,
HdfsClientConfigKeys.HttpClient.RETRY_MAX_ATTEMPTS_DEFAULT);
int failoverSleepBaseMillis = conf.getInt(
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_KEY,
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_BASE_DEFAULT);
HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_KEY,
HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_BASE_DEFAULT);
int failoverSleepMaxMillis = conf.getInt(
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_KEY,
HdfsClientConfigKeys.WebHdfsRetry.FAILOVER_SLEEPTIME_MAX_DEFAULT);
HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_KEY,
HdfsClientConfigKeys.HttpClient.FAILOVER_SLEEPTIME_MAX_DEFAULT);
this.retryPolicy = RetryPolicies
.failoverOnNetworkException(RetryPolicies.TRY_ONCE_THEN_FAIL,

View File

@ -18,13 +18,52 @@
package org.apache.hadoop.hdfs;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import static org.apache.hadoop.fs.CreateFlag.CREATE;
import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.BufferedOutputStream;
import java.io.BufferedReader;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
import java.io.InterruptedIOException;
import java.io.PrintStream;
import java.io.RandomAccessFile;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.net.HttpURLConnection;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.URL;
import java.net.URLConnection;
import java.nio.ByteBuffer;
import java.security.NoSuchAlgorithmException;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import org.apache.commons.io.FileUtils;
import org.apache.commons.logging.Log;
@ -35,24 +74,33 @@
import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclEntryScope;
import org.apache.hadoop.fs.permission.AclEntryType;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@ -75,8 +123,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha
.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
@ -97,21 +144,12 @@
import org.junit.Assume;
import org.mockito.internal.util.reflection.Whitebox;
import java.io.*;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.net.*;
import java.nio.ByteBuffer;
import java.security.NoSuchAlgorithmException;
import java.security.PrivilegedExceptionAction;
import java.util.*;
import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.fs.CreateFlag.*;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
/** Utilities for HDFS tests */
public class DFSTestUtil {
@ -194,7 +232,7 @@ public static void addHAConfiguration(Configuration conf,
}
conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
logicalName), "nn1,nn2");
conf.set(DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "" +
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX +
"." + logicalName,
ConfiguredFailoverProxyProvider.class.getName());
conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
@ -963,8 +1001,8 @@ public static void setFederatedHAConfiguration(MiniDFSCluster cluster,
for (Map.Entry<String, List<String>> entry : nameservices.entrySet()) {
conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX,
entry.getKey()), Joiner.on(",").join(entry.getValue()));
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + entry
.getKey(), ConfiguredFailoverProxyProvider.class.getName());
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "."
+ entry.getKey(), ConfiguredFailoverProxyProvider.class.getName());
}
conf.set(DFSConfigKeys.DFS_NAMESERVICES, Joiner.on(",")
.join(nameservices.keySet()));

View File

@ -17,18 +17,17 @@
*/
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.lang.reflect.Field;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.SocketAddress;
import java.lang.reflect.Field;
import java.net.InetAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.List;
@ -42,13 +41,12 @@
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.namenode.ha.IPFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.hdfs.server.namenode.ha.IPFailoverProxyProvider;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
import org.apache.hadoop.io.retry.FailoverProxyProvider;
import org.apache.hadoop.net.ConnectTimeoutException;
import org.apache.hadoop.net.StandardSocketFactory;
@ -206,7 +204,7 @@ public void testLogicalUriShouldNotHavePorts() {
public void testFailureWithMisconfiguredHaNNs() throws Exception {
String logicalHost = "misconfigured-ha-uri";
Configuration conf = new Configuration();
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalHost,
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalHost,
ConfiguredFailoverProxyProvider.class.getName());
URI uri = new URI("hdfs://" + logicalHost + "/test");
@ -334,7 +332,7 @@ public void testWrappedFailoverProxyProvider() throws Exception {
Configuration config = new HdfsConfiguration(conf);
String logicalName = HATestUtil.getLogicalHostname(cluster);
HATestUtil.setFailoverConfigurations(cluster, config, logicalName);
config.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
config.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
DummyLegacyFailoverProxyProvider.class.getName());
Path p = new Path("hdfs://" + logicalName + "/");
@ -354,7 +352,7 @@ public void testIPFailoverProxyProviderLogicalUri() throws Exception {
// setup the config with the IP failover proxy provider class
Configuration config = new HdfsConfiguration(conf);
URI nnUri = cluster.getURI(0);
config.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." +
config.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." +
nnUri.getHost(),
IPFailoverProxyProvider.class.getName());

View File

@ -875,7 +875,7 @@ public static void namenodeRestartTest(final Configuration conf,
final Path dir = new Path("/testNamenodeRestart");
if (isWebHDFS) {
conf.setBoolean(HdfsClientConfigKeys.WebHdfsRetry.RETRY_POLICY_ENABLED_KEY, true);
conf.setBoolean(HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_KEY, true);
} else {
conf.setBoolean(HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY, true);
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_INTERNAL_NAMESERVICES_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
@ -60,6 +59,7 @@
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -603,7 +603,7 @@ private static Configuration createWebHDFSHAConfiguration(String logicalHostName
conf.set(DFSUtil.addKeySuffixes(
DFS_NAMENODE_HTTP_ADDRESS_KEY, "ns1", "nn2"), nnaddr2);
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalHostName,
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalHostName,
ConfiguredFailoverProxyProvider.class.getName());
return conf;
}

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hdfs.qjournal;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
@ -33,6 +32,7 @@
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
@ -137,7 +137,7 @@ private Configuration initHAConf(URI journalURI, Configuration conf) {
conf.set(DFSConfigKeys.DFS_NAMESERVICES, NAMESERVICE);
conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, NAMESERVICE),
NN1 + "," + NN2);
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + NAMESERVICE,
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + NAMESERVICE,
ConfiguredFailoverProxyProvider.class.getName());
conf.set("fs.defaultFS", "hdfs://" + NAMESERVICE);

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
@ -35,6 +34,7 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
@ -189,7 +189,7 @@ public static void setFailoverConfigurations(Configuration conf,
conf.set(DFSConfigKeys.DFS_NAMESERVICES, logicalName);
conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, logicalName),
nameNodeId1 + "," + nameNodeId2);
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
ConfiguredFailoverProxyProvider.class.getName());
conf.set("fs.defaultFS", "hdfs://" + logicalName);
}

View File

@ -38,6 +38,7 @@
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@ -448,8 +449,7 @@ public void testPipelineRecoveryStress() throws Exception {
// This test triggers rapid NN failovers. The client retry policy uses an
// exponential backoff. This can quickly lead to long sleep times and even
// timeout the whole test. Cap the sleep time at 1s to prevent this.
harness.conf.setInt(DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
1000);
harness.conf.setInt(HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY, 1000);
final MiniDFSCluster cluster = harness.startCluster();
try {

View File

@ -58,8 +58,11 @@
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@ -69,8 +72,6 @@
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@ -201,8 +202,8 @@ private DFSClient genClientWithDummyHandler() throws IOException {
failoverProxyProvider, RetryPolicies
.failoverOnNetworkException(RetryPolicies.TRY_ONCE_THEN_FAIL,
Integer.MAX_VALUE,
DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT,
DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT));
HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT,
HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT));
ClientProtocol proxy = (ClientProtocol) Proxy.newProxyInstance(
failoverProxyProvider.getInterface().getClassLoader(),
new Class[] { ClientProtocol.class }, dummyHandler);