HDFS-8270. create() always retried with hardcoded timeout when file already exists with open lease (Contributed by J.Andreina)

(cherry picked from commit 54f83d9bd9)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
This commit is contained in:
Vinayakumar B 2015-06-03 12:11:46 +05:30
parent a7058caa72
commit 8090a6ee63
3 changed files with 4 additions and 17 deletions

View File

@ -99,6 +99,9 @@ Release 2.7.1 - UNRELEASED
HDFS-8486. DN startup may cause severe data loss (Daryn Sharp via Colin P.
McCabe)
HDFS-8270. create() always retried with hardcoded timeout when file already
exists with open lease (J.Andreina via vinayakumarb)
Release 2.7.0 - 2015-04-20
INCOMPATIBLE CHANGES

View File

@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSClient.Conf;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
@ -68,7 +67,6 @@ import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.io.retry.RetryUtils;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
import org.apache.hadoop.security.SecurityUtil;
@ -425,22 +423,9 @@ public class NameNodeProxies {
if (withRetries) { // create the proxy with retries
RetryPolicy createPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(5,
HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap
= new HashMap<Class<? extends Exception>, RetryPolicy>();
remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
createPolicy);
RetryPolicy methodPolicy = RetryPolicies.retryByRemoteException(
defaultPolicy, remoteExceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap
= new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("create", methodPolicy);
ClientProtocol translatorProxy =
new ClientNamenodeProtocolTranslatorPB(proxy);
return (ClientProtocol) RetryProxy.create(

View File

@ -405,9 +405,8 @@ public class TestFileCreation {
} 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();