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

Arpit Agarwal commented on HDFS-7858:
-------------------------------------

Thanks for updating the patch Arun. The MultiException approach looks like a 
good alternative to refactoring RetryPolicy.

A few comments:
# I didn't understand the call to {{super.performFailover}} in 
{{RequestHedgingProxyProvider#getProxy}}.
# The documentation in HDFSHighAvailabilityWithQJM.md and 
HDFSHighAvailabilityWithNFS.md should be updated as it states _The only 
implementation which currently ships with Hadoop is the 
ConfiguredFailoverProxyProvider_. Okay to do this in a separate Jira.
# Agree with Jing's suggestion to use a {{CompletionService}}.

Also we should file a task to make {{RequestHedgingProxyProvider}} the default 
eventually.

Nitpicks:
# {{getDelayMillis}} javadoc is wrong.
# {{successfullproxy}} should be {{successfulproxy}}.
# _new LinkedList<RetryAction>_ - explicit type argument redundant.
# _static interface ProxyFactory_ - static is redundant.

> Improve HA Namenode Failover detection on the client
> ----------------------------------------------------
>
>                 Key: HDFS-7858
>                 URL: https://issues.apache.org/jira/browse/HDFS-7858
>             Project: Hadoop HDFS
>          Issue Type: Improvement
>            Reporter: Arun Suresh
>            Assignee: Arun Suresh
>              Labels: BB2015-05-TBR
>         Attachments: HDFS-7858.1.patch, HDFS-7858.2.patch, HDFS-7858.2.patch, 
> HDFS-7858.3.patch, HDFS-7858.4.patch, HDFS-7858.5.patch, HDFS-7858.6.patch
>
>
> In an HA deployment, Clients are configured with the hostnames of both the 
> Active and Standby Namenodes.Clients will first try one of the NNs 
> (non-deterministically) and if its a standby NN, then it will respond to the 
> client to retry the request on the other Namenode.
> If the client happens to talks to the Standby first, and the standby is 
> undergoing some GC / is busy, then those clients might not get a response 
> soon enough to try the other NN.
> Proposed Approach to solve this :
> 1) Since Zookeeper is already used as the failover controller, the clients 
> could talk to ZK and find out which is the active namenode before contacting 
> it.
> 2) Long-lived DFSClients would have a ZK watch configured which fires when 
> there is a failover so they do not have to query ZK everytime to find out the 
> active NN
> 2) Clients can also cache the last active NN in the user's home directory 
> (~/.lastNN) so that short-lived clients can try that Namenode first before 
> querying ZK



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to