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

Todd Lipcon commented on HDFS-1973:
-----------------------------------

bq. Sure, but the whole of DFSClient is annotated @InterfaceAudience.Private. 
You still think we should keep it around?

Yes, because unfortunately the Private annotation was added after the 0.20 
release. If it's problematic to keep around, we don't have to, but it seemed 
easy enough to maintain for now.

----
{code}
+  @Override
+  public synchronized void performFailover(Object currentProxy) {
+    if (proxies.get(currentProxyIndex) != currentProxy) {
+      currentProxyIndex = (currentProxyIndex + 1) % proxies.size();
+    }
+  }
{code}
This code confuses me -- isn't {{currentProxy}} a proxy object, whereas 
{{proxies.get(...)}} is an {{AddressRpcProxyPair}}? Which is to say they're 
always un-equal?

                
> HA: HDFS clients must handle namenode failover and switch over to the new 
> active namenode.
> ------------------------------------------------------------------------------------------
>
>                 Key: HDFS-1973
>                 URL: https://issues.apache.org/jira/browse/HDFS-1973
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>            Reporter: Suresh Srinivas
>            Assignee: Aaron T. Myers
>         Attachments: HDFS-1973-HDFS-1623.patch, HDFS-1973-HDFS-1623.patch, 
> hdfs-1973.0.patch
>
>
> During failover, a client must detect the current active namenode failure and 
> switch over to the new active namenode. The switch over might make use of IP 
> failover or some thing more elaborate such as zookeeper to discover the new 
> active.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to