Skip to content

Commit

Permalink
HDFS-8403. Eliminate retries in TestFileCreation#testOverwriteOpenFor…
Browse files Browse the repository at this point in the history
…Write. Contributed by Arpit Agarwal.
  • Loading branch information
Haohui Mai committed May 16, 2015
1 parent e5afac5 commit ac742c7
Show file tree
Hide file tree
Showing 5 changed files with 51 additions and 4 deletions.
3 changes: 3 additions & 0 deletions hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
Expand Up @@ -775,6 +775,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8371. Fix test failure in TestHdfsConfigFields for spanreceiver
properties. (Ray Chiang via aajisaka)

HDFS-8403. Eliminate retries in TestFileCreation
#testOverwriteOpenForWrite. (Arpit Agarwal via wheat9)

Release 2.7.1 - UNRELEASED

INCOMPATIBLE CHANGES
Expand Down
Expand Up @@ -341,10 +341,13 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
this.namenode = rpcNamenode;
dtService = null;
} else {
boolean noRetries = conf.getBoolean(
DFSConfigKeys.DFS_CLIENT_TEST_NO_PROXY_RETRIES,
DFSConfigKeys.DFS_CLIENT_TEST_NO_PROXY_RETRIES_DEFAULT);
Preconditions.checkArgument(nameNodeUri != null,
"null URI");
proxyInfo = NameNodeProxies.createProxy(conf, nameNodeUri,
ClientProtocol.class, nnFallbackToSimpleAuth);
ClientProtocol.class, nnFallbackToSimpleAuth, !noRetries);
this.dtService = proxyInfo.getDelegationTokenService();
this.namenode = proxyInfo.getProxy();
}
Expand Down
Expand Up @@ -20,6 +20,7 @@

import java.util.concurrent.TimeUnit;

import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
Expand Down Expand Up @@ -993,6 +994,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
// For testing NN retry cache, we can set this property with positive value.
public static final String DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY = "dfs.client.test.drop.namenode.response.number";
public static final int DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT = 0;

// Create a NN proxy without retries for testing.
@VisibleForTesting
public static final String DFS_CLIENT_TEST_NO_PROXY_RETRIES =
"dfs.client.test.no.proxy.retries";
@VisibleForTesting
public static final boolean DFS_CLIENT_TEST_NO_PROXY_RETRIES_DEFAULT = false;

public static final String DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY =
"dfs.client.slow.io.warning.threshold.ms";
public static final long DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT = 30000;
Expand Down
Expand Up @@ -161,14 +161,40 @@ public static <T> ProxyAndInfo<T> createProxy(Configuration conf,
public static <T> ProxyAndInfo<T> createProxy(Configuration conf,
URI nameNodeUri, Class<T> xface, AtomicBoolean fallbackToSimpleAuth)
throws IOException {
return createProxy(conf, nameNodeUri, xface, fallbackToSimpleAuth, true);
}

/**
* Creates the namenode proxy with the passed protocol. This will handle
* creation of either HA- or non-HA-enabled proxy objects, depending upon
* if the provided URI is a configured logical URI.
*
* @param conf the configuration containing the required IPC
* properties, client failover configurations, etc.
* @param nameNodeUri the URI pointing either to a specific NameNode
* or to a logical nameservice.
* @param xface the IPC interface which should be created
* @param fallbackToSimpleAuth set to true or false during calls to
* indicate if a secure client falls back to simple auth
* @param withRetries certain interfaces have a non-standard retry policy
* @return an object containing both the proxy and the associated
* delegation token service it corresponds to
* @throws IOException if there is an error creating the proxy
**/
@SuppressWarnings("unchecked")
public static <T> ProxyAndInfo<T> createProxy(Configuration conf,
URI nameNodeUri, Class<T> xface, AtomicBoolean fallbackToSimpleAuth,
boolean withRetries)
throws IOException {
AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
createFailoverProxyProvider(conf, nameNodeUri, xface, true,
fallbackToSimpleAuth);

if (failoverProxyProvider == null) {
// Non-HA case
return createNonHAProxy(conf, NameNode.getAddress(nameNodeUri), xface,
UserGroupInformation.getCurrentUser(), true, fallbackToSimpleAuth);
UserGroupInformation.getCurrentUser(), withRetries,
fallbackToSimpleAuth);
} else {
// HA case
DfsClientConf config = new DfsClientConf(conf);
Expand Down
Expand Up @@ -65,6 +65,7 @@
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
Expand All @@ -73,6 +74,7 @@
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
Expand Down Expand Up @@ -377,6 +379,11 @@ public void testOverwriteOpenForWrite() throws Exception {
Configuration conf = new HdfsConfiguration();
SimulatedFSDataset.setFactory(conf);
conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);

// Force NameNodeProxies' createNNProxyWithClientProtocol to give
// up file creation after one failure.
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_TEST_NO_PROXY_RETRIES, true);

final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
FileSystem fs = cluster.getFileSystem();

Expand Down Expand Up @@ -405,9 +412,8 @@ public FileSystem run() throws Exception {
} catch (IOException abce) {
GenericTestUtils.assertExceptionContains("Failed to CREATE_FILE", abce);
}
// NameNodeProxies' createNNProxyWithClientProtocol has 5 retries.
assertCounter("AlreadyBeingCreatedExceptionNumOps",
6L, getMetrics(metricsName));
1L, getMetrics(metricsName));
FSDataOutputStream stm2 = fs2.create(p, true);
stm2.write(2);
stm2.close();
Expand Down

0 comments on commit ac742c7

Please sign in to comment.