[ https://issues.apache.org/jira/browse/HDFS-7858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15360989#comment-15360989 ]
He Xiaoqiao commented on HDFS-7858: ----------------------------------- hi [~asuresh] when i patch this to 2.7.1, it throws some exception when submit job as following: {quote} 2016-07-01 17:45:37,497 WARN [pool-9-thread-2] org.apache.hadoop.ipc.Client: Exception encountered while connecting to the server : java.nio.channels.ClosedByInterruptException 2016-07-01 17:45:37,542 WARN [pool-10-thread-2] org.apache.hadoop.ipc.Client: Exception encountered while connecting to the server : java.nio.channels.ClosedByInterruptException 2016-07-01 17:45:37,571 INFO [main] org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler: Emitting job history data to the timeline server is not enabled 2016-07-01 17:45:37,572 WARN [pool-11-thread-2] org.apache.hadoop.ipc.Client: Exception encountered while connecting to the server : java.nio.channels.ClosedByInterruptException 2016-07-01 17:45:37,573 INFO [main] org.apache.hadoop.mapreduce.v2.app.MRAppMaster: Recovery is enabled. Will try to recover from previous life on best effort basis. 2016-07-01 17:45:37,633 INFO [main] org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils: Default file system [viewfs://ha/] 2016-07-01 17:45:37,698 INFO [main] org.apache.hadoop.mapreduce.v2.app.MRAppMaster: Previous history file is at viewfs://ha/hadoop-yarn/staging/yarn/.staging/job_1467365572539_3212/job_1467365572539_3212_1.jhist 2016-07-01 17:45:37,713 WARN [main] org.apache.hadoop.hdfs.server.namenode.ha.RequestHedgingProxyProvider: Invocation returned exception on [nn1host/ip:port] 2016-07-01 17:45:37,716 WARN [pool-12-thread-2] org.apache.hadoop.ipc.Client: Exception encountered while connecting to the server : org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ipc.StandbyException): Operation category READ is not supported in state standby 2016-07-01 17:45:37,717 WARN [main] org.apache.hadoop.hdfs.server.namenode.ha.RequestHedgingProxyProvider: Invocation returned exception on [nn2host/ip:port] 2016-07-01 17:45:37,725 WARN [main] org.apache.hadoop.mapreduce.v2.app.MRAppMaster: Unable to parse prior job history, aborting recovery MultiException[{java.util.concurrent.ExecutionException: java.lang.reflect.InvocationTargetException, java.util.concurrent.ExecutionException: java.lang.reflect.InvocationTargetException, }] at org.apache.hadoop.hdfs.server.namenode.ha.RequestHedgingProxyProvider$RequestHedgingInvocationHandler.invoke(RequestHedgingProxyProvider.java:133) at com.sun.proxy.$Proxy10.getBlockLocations(Unknown Source) 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.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:252) at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104) at com.sun.proxy.$Proxy10.getBlockLocations(Unknown Source) at org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1226) at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1213) at org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1201) at org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:303) at org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:269) at org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:261) at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1526) at org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:303) at org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:299) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:299) at org.apache.hadoop.fs.FilterFileSystem.open(FilterFileSystem.java:161) at org.apache.hadoop.fs.viewfs.ChRootedFileSystem.open(ChRootedFileSystem.java:257) at org.apache.hadoop.fs.viewfs.ViewFileSystem.open(ViewFileSystem.java:423) at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:788) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.getPreviousJobHistoryStream(MRAppMaster.java:1199) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.parsePreviousJobHistory(MRAppMaster.java:1203) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.processRecovery(MRAppMaster.java:1175) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.serviceStart(MRAppMaster.java:1039) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster$4.run(MRAppMaster.java:1519) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.initAndStartAppMaster(MRAppMaster.java:1515) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.main(MRAppMaster.java:1448) {quote} is this out of expectation? > 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 > Components: hdfs-client > Reporter: Arun Suresh > Assignee: Arun Suresh > Labels: BB2015-05-TBR > Fix For: 2.8.0 > > Attachments: HDFS-7858.1.patch, HDFS-7858.10.patch, > HDFS-7858.10.patch, HDFS-7858.11.patch, HDFS-7858.12.patch, > HDFS-7858.13.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, HDFS-7858.7.patch, > HDFS-7858.8.patch, HDFS-7858.9.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) Use hedged RPCs to simultaneously call multiple configured NNs to decide > which is the active Namenode. > 2) Subsequent calls, will invoke the previously successful NN. > 3) On failover of the currently active NN, the remaining NNs will be invoked > to decide which is the new active -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org