HDFS-8270. create() always retried with hardcoded timeout when file already exists with open lease (Contributed by J.Andreina)
(cherry picked from commit54f83d9bd9
) 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 (cherry picked from commit8090a6ee63
) Conflicts: hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (cherry picked from commit 04a7000c8d955a134d86967f4b4622ea920e1ea7)
This commit is contained in:
parent
ae0fac3efa
commit
a2434dcf2a
|
@ -150,6 +150,9 @@ Release 2.6.1 - UNRELEASED
|
|||
HDFS-7609. Avoid retry cache collision when Standby NameNode loading edits.
|
||||
(Ming Ma via jing9)
|
||||
|
||||
HDFS-8270. create() always retried with hardcoded timeout when file already
|
||||
exists with open lease (J.Andreina via vinayakumarb)
|
||||
|
||||
Release 2.6.0 - 2014-11-18
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -408,9 +408,8 @@ public class TestFileCreation {
|
|||
GenericTestUtils.assertExceptionContains("already being created by",
|
||||
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();
|
||||
|
|
Loading…
Reference in New Issue