hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Daryn Sharp (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-6475) WebHdfs clients fail without retry because incorrect handling of StandbyException
Date Fri, 20 Jun 2014 18:17:27 GMT

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

Daryn Sharp commented on HDFS-6475:
-----------------------------------

I don't think {{getTrueCause}} is necessary anymore and can be removed.  It was born when
{{SaslDigestCallbackHandler}} used to call {{SecretManager#retrievePassword}} which could
only throw {{InvalidToken}}.  A standby/retriable/etc exception had to be tunneled out via
an {{InvalidToken}} cause.  Now there's apparently a {{SecretManager#retriableRetrievePassword}}
that can throw those exceptions directly.

The problem comes down to an abuse of the injectable {{UserProvider}} to do authentication
instead of via an authentication filter...  Perhaps {{SecurityException}} is the only exception
for which the cause needs to be extracted.

> WebHdfs clients fail without retry because incorrect handling of StandbyException
> ---------------------------------------------------------------------------------
>
>                 Key: HDFS-6475
>                 URL: https://issues.apache.org/jira/browse/HDFS-6475
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: ha, webhdfs
>    Affects Versions: 2.4.0
>            Reporter: Yongjun Zhang
>            Assignee: Yongjun Zhang
>         Attachments: HDFS-6475.001.patch, HDFS-6475.002.patch, HDFS-6475.003.patch, HDFS-6475.003.patch,
HDFS-6475.004.patch, HDFS-6475.005.patch, HDFS-6475.006.patch
>
>
> With WebHdfs clients connected to a HA HDFS service, the delegation token is previously
initialized with the active NN.
> When clients try to issue request, the NN it contacts is stored in a map returned by
DFSUtil.getNNServiceRpcAddresses(conf). And the client contact the NN based on the order,
so likely the first one it runs into is StandbyNN. If the StandbyNN doesn't have the updated
client crediential, it will throw a s SecurityException that wraps StandbyException.
> The client is expected to retry another NN, but due to the insufficient handling of SecurityException
mentioned above, it failed.
> Example message:
> {code}
> {RemoteException={message=Failed to obtain user group information: org.apache.hadoop.security.token.SecretManager$InvalidToken:
StandbyException, javaCl
> assName=java.lang.SecurityException, exception=SecurityException}}
> org.apache.hadoop.ipc.RemoteException(java.lang.SecurityException): Failed to obtain
user group information: org.apache.hadoop.security.token.SecretManager$InvalidToken: StandbyException
>         at org.apache.hadoop.hdfs.web.JsonUtil.toRemoteException(JsonUtil.java:159)
>         at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.validateResponse(WebHdfsFileSystem.java:325)
>         at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.access$700(WebHdfsFileSystem.java:107)
>         at org.apache.hadoop.hdfs.web.WebHdfsFileSystem$AbstractRunner.getResponse(WebHdfsFileSystem.java:635)
>         at org.apache.hadoop.hdfs.web.WebHdfsFileSystem$AbstractRunner.run(WebHdfsFileSystem.java:542)
>         at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.run(WebHdfsFileSystem.java:431)
>         at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.getHdfsFileStatus(WebHdfsFileSystem.java:685)
>         at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.getFileStatus(WebHdfsFileSystem.java:696)
>         at kclient1.kclient$1.run(kclient.java:64)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:356)
>         at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1528)
>         at kclient1.kclient.main(kclient.java:58)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at org.apache.hadoop.util.RunJar.main(RunJar.java:212)
> {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message