[
https://issues.apache.org/jira/browse/HDFS-14655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16924632#comment-16924632
]
Ayush Saxena commented on HDFS-14655:
-------------------------------------
Thanx [~vagarychen] [~xkrogen] for the discussion. I agree in general as Chen
mentioned parallelism will not pitch in general, as we are also canceling the
threads too. But still in extreme cases, there might be chances where it may be
usefull. I have uploaded v2 with a config. Using the {{newFixedThreadPool}}.
Give a check.
> [SBN Read] Namenode crashes if one of The JN is down
> ----------------------------------------------------
>
> Key: HDFS-14655
> URL: https://issues.apache.org/jira/browse/HDFS-14655
> Project: Hadoop HDFS
> Issue Type: Bug
> Reporter: Harshakiran Reddy
> Assignee: Ayush Saxena
> Priority: Major
> Attachments: HDFS-14655-01.patch, HDFS-14655-02.patch,
> HDFS-14655.poc.patch
>
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO | Logger channel (from parallel executor) to
> XXXXXXX/XXXXXXX | Retrying connect to server: XXXXXXX/XXXXXXX. Already tried
> 9 time(s); retry policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10,
> sleepTime=1000 MILLISECONDS) | Client.java:975
> 2019-07-04 17:35:54,087 | FATAL | Edit log tailer | Unknown error encountered
> while tailing edits. Shutting down standby NN. | EditLogTailer.java:474
> java.lang.OutOfMemoryError: unable to create new native thread
> at java.lang.Thread.start0(Native Method)
> at java.lang.Thread.start(Thread.java:717)
> at
> java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:957)
> at
> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1378)
> at
> com.google.common.util.concurrent.MoreExecutors$ListeningDecorator.execute(MoreExecutors.java:440)
> at
> com.google.common.util.concurrent.AbstractListeningExecutorService.submit(AbstractListeningExecutorService.java:56)
> at
> org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel.getJournaledEdits(IPCLoggerChannel.java:565)
> at
> org.apache.hadoop.hdfs.qjournal.client.AsyncLoggerSet.getJournaledEdits(AsyncLoggerSet.java:272)
> at
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager.selectRpcInputStreams(QuorumJournalManager.java:533)
> at
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager.selectInputStreams(QuorumJournalManager.java:508)
> at
> org.apache.hadoop.hdfs.server.namenode.JournalSet.selectInputStreams(JournalSet.java:275)
> at
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.selectInputStreams(FSEditLog.java:1681)
> at
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.selectInputStreams(FSEditLog.java:1714)
> at
> org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer.doTailEdits(EditLogTailer.java:307)
> at
> org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.doWork(EditLogTailer.java:460)
> at
> org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.access$300(EditLogTailer.java:410)
> at
> org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread$1.run(EditLogTailer.java:427)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:360)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1709)
> at
> org.apache.hadoop.security.SecurityUtil.doAsLoginUserOrFatal(SecurityUtil.java:483)
> at
> org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer$EditLogTailerThread.run(EditLogTailer.java:423)
> 2019-07-04 17:35:54,112 | INFO | Edit log tailer | Exiting with status 1:
> java.lang.OutOfMemoryError: unable to create new native thread |
> ExitUtil.java:210
> {noformat}
--
This message was sent by Atlassian Jira
(v8.3.2#803003)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]