[ 
https://issues.apache.org/jira/browse/HDFS-15467?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17228899#comment-17228899
 ] 

Aihua Xu edited comment on HDFS-15467 at 11/10/20, 1:33 AM:
------------------------------------------------------------

[~csun] In ObserverReadProxyProvider, the FailoverProxyProvider (failoverProxy) 
for active/standby namenode failover doesn't seem to have retry logic. When 
msync() is called against failoverProxy, it could fail when it's reaching out a 
standby namenode. The exception is thrown to the retry logic of 
ObserverReadProxyProvider to handle (see the stack trace below). Is this by 
design? Logically seems FailoverProxyProvider should also have retry around it 
as well like:

{{{{DfsClientConf config = new DfsClientConf(conf);}}}}
{{ClientProtocol proxy = (ClientProtocol) RetryProxy.create(xface,}}
{{failoverProxyProvider,}}
{{RetryPolicies.failoverOnNetworkException(}}
{{RetryPolicies.TRY_ONCE_THEN_FAIL, config.getMaxFailoverAttempts(),}}
{{config.getMaxRetryAttempts(), config.getFailoverSleepBaseMillis(),}}
{{config.getFailoverSleepMaxMillis()));}}
{quote}20/10/29 04:22:33 INFO retry.RetryInvocationHandler: Exception while 
invoking $Proxy5.getFileInfo over 
[hadoopetanamenode01-dca1.prod.uber.internal/10.22.3.137:8020,hadoopetanamenode02-dca1.prod.uber.internal/10.18.6.167:8020,hadoopetaobserver01-dca1.prod.uber.internal/10.14.137.154:8020]
 after 1 failover attempts. Trying to failover after sleeping for 693ms.
 org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ipc.StandbyException): 
Operation category WRITE is not supported in state standby. Visit 
[http://t.uber.com/hdfs_faq]
 at 
org.apache.hadoop.hdfs.server.namenode.ha.StandbyState.checkOperation(StandbyState.java:108)
 at 
org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.checkOperation(NameNode.java:1942)
 at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkOperation(FSNamesystem.java:1387)
 at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.msync(NameNodeRpcServer.java:1318)
 at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.msync(ClientNamenodeProtocolServerSideTranslatorPB.java:1617)
 at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:508)
 at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1034)
 at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:930)
 at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:865)
 at java.security.AccessController.doPrivileged(Native Method)
 at javax.security.auth.Subject.doAs(Subject.java:422)
 at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836)
 at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2726)

at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1524)
 at org.apache.hadoop.ipc.Client.call(Client.java:1470)
 at org.apache.hadoop.ipc.Client.call(Client.java:1369)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:231)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:117)
 at com.sun.proxy.$Proxy15.msync(Unknown Source)
 at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.msync(ClientNamenodeProtocolTranslatorPB.java:1634)
 at 
org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.initializeMsync(ObserverReadProxyProvider.java:350)
 at 
org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.access$600(ObserverReadProxyProvider.java:69)
 at 
org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider$ObserverReadInvocationHandler.invoke(ObserverReadProxyProvider.java:427)
 at com.sun.proxy.$Proxy5.getFileInfo(Unknown Source)
 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
 at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
 at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:498)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:409)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:163)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:155)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:346)
 at com.sun.proxy.$Proxy5.getFileInfo(Unknown Source)
 at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1700)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem$27.doCall(DistributedFileSystem.java:1439)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem$27.doCall(DistributedFileSystem.java:1436)
 at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1436)
 at org.apache.hadoop.fs.Globber.getFileStatus(Globber.java:64)
 at org.apache.hadoop.fs.Globber.doGlob(Globber.java:269)
 at org.apache.hadoop.fs.Globber.glob(Globber.java:148)
 at org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:1686)
 at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:326)
 at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:245)
 at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:228)
 at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103)
 at org.apache.hadoop.fs.shell.Command.run(Command.java:175)
 at org.apache.hadoop.fs.FsShell.run(FsShell.java:315)
 at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
 at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
 at org.apache.hadoop.fs.FsShell.main(FsShell.java:378)
{quote}


was (Author: aihuaxu):
[~csun] In ObserverReadProxyProvider, the FailoverProxyProvider (failoverProxy) 
for active/standby namenode failover doesn't seem to have retry logic. When 
msync() is called against failoverProxy, it could fail when it's reaching out a 
standby namenode. The exception is thrown to the retry logic of 
ObserverReadProxyProvider to handle (see the stack trace below). Is this by 
design? Logically seems FailoverProxyProvider should also have retry around it 
as well like:

{{DfsClientConf config = new DfsClientConf(conf);}}
{{ ClientProtocol proxy = (ClientProtocol) RetryProxy.create(xface,}}
{{ failoverProxyProvider,}}
{{ RetryPolicies.failoverOnNetworkException(}}
{{ RetryPolicies.TRY_ONCE_THEN_FAIL, config.getMaxFailoverAttempts(),}}
{{ config.getMaxRetryAttempts(), config.getFailoverSleepBaseMillis(),}}
{{ config.getFailoverSleepMaxMillis()));}}
{quote}20/10/29 04:22:33 INFO retry.RetryInvocationHandler: Exception while 
invoking $Proxy5.getFileInfo over 
[hadoopetanamenode01-dca1.prod.uber.internal/10.22.3.137:8020,hadoopetanamenode02-dca1.prod.uber.internal/10.18.6.167:8020,hadoopetaobserver01-dca1.prod.uber.internal/10.14.137.154:8020]
 after 1 failover attempts. Trying to failover after sleeping for 693ms.
 org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ipc.StandbyException): 
Operation category WRITE is not supported in state standby. Visit 
[http://t.uber.com/hdfs_faq]
 at 
org.apache.hadoop.hdfs.server.namenode.ha.StandbyState.checkOperation(StandbyState.java:108)
 at 
org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.checkOperation(NameNode.java:1942)
 at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkOperation(FSNamesystem.java:1387)
 at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.msync(NameNodeRpcServer.java:1318)
 at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.msync(ClientNamenodeProtocolServerSideTranslatorPB.java:1617)
 at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:508)
 at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1034)
 at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:930)
 at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:865)
 at java.security.AccessController.doPrivileged(Native Method)
 at javax.security.auth.Subject.doAs(Subject.java:422)
 at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836)
 at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2726)

at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1524)
 at org.apache.hadoop.ipc.Client.call(Client.java:1470)
 at org.apache.hadoop.ipc.Client.call(Client.java:1369)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:231)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:117)
 at com.sun.proxy.$Proxy15.msync(Unknown Source)
 at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.msync(ClientNamenodeProtocolTranslatorPB.java:1634)
 at 
org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.initializeMsync(ObserverReadProxyProvider.java:350)
 at 
org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.access$600(ObserverReadProxyProvider.java:69)
 at 
org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider$ObserverReadInvocationHandler.invoke(ObserverReadProxyProvider.java:427)
 at com.sun.proxy.$Proxy5.getFileInfo(Unknown Source)
 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
 at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
 at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:498)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:409)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:163)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:155)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:346)
 at com.sun.proxy.$Proxy5.getFileInfo(Unknown Source)
 at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1700)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem$27.doCall(DistributedFileSystem.java:1439)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem$27.doCall(DistributedFileSystem.java:1436)
 at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1436)
 at org.apache.hadoop.fs.Globber.getFileStatus(Globber.java:64)
 at org.apache.hadoop.fs.Globber.doGlob(Globber.java:269)
 at org.apache.hadoop.fs.Globber.glob(Globber.java:148)
 at org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:1686)
 at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:326)
 at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:245)
 at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:228)
 at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103)
 at org.apache.hadoop.fs.shell.Command.run(Command.java:175)
 at org.apache.hadoop.fs.FsShell.run(FsShell.java:315)
 at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
 at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
 at org.apache.hadoop.fs.FsShell.main(FsShell.java:378)
{quote}

> ObserverReadProxyProvider should skip logging first failover from each proxy
> ----------------------------------------------------------------------------
>
>                 Key: HDFS-15467
>                 URL: https://issues.apache.org/jira/browse/HDFS-15467
>             Project: Hadoop HDFS
>          Issue Type: Task
>            Reporter: Hanisha Koneru
>            Assignee: Aihua Xu
>            Priority: Major
>
> After HADOOP-17116, \{{RetryInvocationHandler}} skips logging the first 
> failover INFO message from each proxy. But {{ObserverReadProxyProvider}} uses 
> {{combinedProxy}} object which combines all proxies into one and assigns 
> {{combinedInfo}} as the ProxyInfo.
> {noformat}
> ObserverReadProxyProvider# Lines 197-207:
> for (int i = 0; i < nameNodeProxies.size(); i++) {
>   if (i > 0) {
>     combinedInfo.append(",");
>   }
>   combinedInfo.append(nameNodeProxies.get(i).proxyInfo);
> }
> combinedInfo.append(']');
> T wrappedProxy = (T) Proxy.newProxyInstance(
>     ObserverReadInvocationHandler.class.getClassLoader(),
>     new Class<?>[] {xface}, new ObserverReadInvocationHandler());
> combinedProxy = new ProxyInfo<>(wrappedProxy, 
> combinedInfo.toString()){noformat}
> {{RetryInvocationHandler}} depends on the {{ProxyInfo}} to differentiate 
> between proxies while checking if failover from that proxy happened before. 
> And since combined proxy has only 1 proxy, HADOOP-17116 doesn't work on 
> {{ObserverReadProxyProvider.}}It would need to handled separately.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to