[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-08-08 Thread Konstantin Shvachko (JIRA)


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

Konstantin Shvachko commented on HDFS-14655:


Hey guys, discussed this with Chen. It seems that we need a pool of only 3 
threads, which can be reused for each iteration of tailing. Here 3 = number of 
Journal Nodes. Creating threads with such high frequency seems to be expensive 
in all aspects. How hard would it be to make this change?

> SBN : 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.poc.patch
>
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-22 Thread Erik Krogen (JIRA)


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

Erik Krogen commented on HDFS-14655:


Great discussion here. [~ayushtkn], particularly good call on the issue that 
cancelling is not fully sufficient to fix this issue.

I agree that calling cancel + limiting the size of the {{parallelExecutor}} 
seems to be a good approach. That executor is scoped to a single JN, so a limit 
will not affect other JNs if one is running slowly. Plus, the 
{{parallelExecutor}} is only used by {{getJournaledEdits}} and 
{{getEditLogManifest}} (others use the {{singleThreadExecutor}}) so no other 
operations besides edit log tailing should be affected. It seems we'll need to 
use {{new ThreadPoolExecutor()}} directly instead of the {{Executors}} 
convenience method.

You said that many {{InterruptedException}} instances are being logged, is 
there any way we can suppress them? Where are they logged from?

> SBN : 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
>Priority: Major
> Attachments: HDFS-14655.poc.patch
>
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-18 Thread Ayush Saxena (JIRA)


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

Ayush Saxena commented on HDFS-14655:
-

Thanx [~vagarychen]. The solution that we have is, We too cancel the remaining 
thread(though it doesn't immediately clears up), But better than retrying 10 
times.
 And a change here in the {{IPCLoggerChannel}}
{code:java}
  protected ExecutorService createParallelExecutor() {
return Executors.newCachedThreadPool(
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("Logger channel (from parallel executor) to " + addr)
.setUncaughtExceptionHandler(
UncaughtExceptionHandlers.systemExit())
.build());
  }
{code}
Change this thread pool with one which we can limit the maximum threads. This 
way the OOM got solved but not sure this is the best thing to do. Or if there 
is better way to do it.

Will see we can get some more opinions here.

[~shv] [~xkrogen] [~ste...@apache.org] Any pointers?

> SBN : 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
>Priority: Major
> Attachments: HDFS-14655.poc.patch
>
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-17 Thread Chen Liang (JIRA)


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

Chen Liang commented on HDFS-14655:
---

Thanks for sharing the details [~ayushtkn]! I also wondered if just calling 
cancel() is enough to reclaim resources. Then I guess unless we have a way to 
clean up failed threads fast enough (which I doubt), we will have to have a cap 
on how much resources can be allocated on failed JNs, i.e. have a cap on how 
many threads can be created for it in some manner. 

About the issue of killing the slow ones as well though, I think even now,  it 
can already be happening, i.e. skipping responses if some response is slow. The 
contract of requiring only a quorum, not all, to response already suggests 
allowing slow ones to be missed. Also, given that we do tail-edits very 
frequently in this case, we should be catching very fast and if a JN is slow, 
it might already be consistently skipped, so I'm not too much concerned about 
this.

Since it seems you have been actively working on this and have done some great 
investigation already. Would you mind picking up this Jira :)? I will be 
closely watching as well.

> SBN : 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
>Priority: Major
> Attachments: HDFS-14655.poc.patch
>
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-17 Thread Ayush Saxena (JIRA)


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

Ayush Saxena commented on HDFS-14655:
-

Thanx [~vagarychen] we too tried with just the approach of cancelling the 
outstanding calls before we move hand. 
 Firstly when the {{dfs.ha.tail-edits.period}} period is 0. With this approach, 
the problem doesn't get solved. As when we call, we don't actually kill the 
thread, but just call interrupt, and when the connection thread comes in the 
next iteration and sees it being interrupted, then it stops. So this whole 
process also has some latency. With 0 as configured value, The pace at which 
the thread is removed by cancel is outsmarted by the tailing frequency.
 Secondly, As you said, for normal scenario also, this shall be happen, when 
majority comes fast and others were just suppose to come(We can't expect 
practically for all responses together), so we will land up killing them too. 
And this is little noise too as the LOGs shall also be filled with 
InterruptedException.

 

> SBN : 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
>Priority: Major
> Attachments: HDFS-14655.poc.patch
>
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-17 Thread Chen Liang (JIRA)


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

Chen Liang commented on HDFS-14655:
---

Post a POC patch with the idea of cancelling unfinished calls after getting 
quorum for read. I marked as POC as I still haven't found a good way to verify 
this fix the issue, since I don't have the setup to reproduce the issue at the 
moment. Still share the patch though, to hopefully get some comments/insights.

Basically, the idea of the POC patch is if a read has got a quorum response, 
call Future#cancel() on the remaining tasks. One tricky trade-off is that if it 
is not a actually failed read, just a slow one, it may also got killed here. 
I'm a bit concerned that this idea could mean we effectively always read from 
just 2 JNs.

I added a POC unit test, please be aware this is NOT a finalized test, just to 
illustrate the idea. I log the total number of threads. In my local run, 
without the cancelling change, the thread count started at 53, always end up at 
~210 threads. With the change, the thread count started at 53, always end up at 
~70. (In this test though, for there was no retry connecting, always just 
connection refused).

> SBN : 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
>Priority: Major
> Attachments: HDFS-14655.poc.patch
>
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-17 Thread Chen Liang (JIRA)


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

Chen Liang commented on HDFS-14655:
---

Thanks [~ayushtkn]. I'm also thinking of cancelling outstanding calls if we 
have got response from a quorum. For restricting number of threads though, the 
outstanding calls may still pile up and use up the threads counts. So we will 
need to restrict number of threads with some strategy, e.g. restricting number 
of threads on per JN level.

> SBN : 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
>Priority: Major
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-17 Thread Ayush Saxena (JIRA)


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

Ayush Saxena commented on HDFS-14655:
-

Thanx [~vagarychen], If you check the trace, the first line shows that too. 
(Well if you require some more logs on that, I will try arrange more for that).

That is the general retry logic for connection. Check in Client.java 
handleConnectionFailure(..),

What I think is, May be we can change the {{createParallelExecutor()}} in 
{{IPCLoggerChannel.java}} to restrict the number of threads..

Or even before returning once we get the quorum, We should try kill the other 
threads and then return. Or may be both, Or something else!!!

> SBN : 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
>Priority: Major
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-17 Thread Chen Liang (JIRA)


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

Chen Liang commented on HDFS-14655:
---

Thanks [~ayushtkn]. Then looks like even without observer read, as long as 
{{dfs.ha.tail-edits.period}} is set low enough, due to we require only a 
quorum, not all standby to response to proceed, when next edit tailing call 
happens, there can be connections from previous call still pending, piling up 
more outstanding calls. But I haven't found where the "the third JN keeps on 
trying 10 times on ConnectException" is happening. Would you mind elaborating 
on this part a bit? What ConnectException exactly was it?

> SBN : 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
>Priority: Major
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-16 Thread Ayush Saxena (JIRA)


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

Ayush Saxena commented on HDFS-14655:
-

Thanx [~vagarychen]

Yes, It affected all the observers.

In standby NN we didn't enable tailling edits so the standby namenode were not 
affected, they stayed intact. But when the ONN was restarted after the crash, 
it starts in Standby state. so it was crashing in that state too.

The problem is with tailing. It can happen with standby too, if edit log 
tailing is enabled. But we just had it enabled in our Observer node only, To 
prevent stale reads

 

> SBN : 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
>Priority: Major
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The JN is down

2019-07-16 Thread Chen Liang (JIRA)


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

Chen Liang commented on HDFS-14655:
---

Thanks for reporting this and really appreciate the investigation [~ayushtkn]. 
So just to clarify based on the description. NN that crashed is Observer node, 
or other Standby as well? Were all the observer nodes died eventually? Because 
based on what you mentioned, looks like chances are all the observers will be 
run into this at some point.

> SBN : 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
>Priority: Major
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14655) SBN : Namenode crashes if one of The jN is down

2019-07-15 Thread Ayush Saxena (JIRA)


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

Ayush Saxena commented on HDFS-14655:
-

For Observer read, The edit log tail period is set to 0, 
(dfs.ha.tail-edits.period)
There are three Journal nodes, the tailing process succeeds if it is able to 
fetch the response from majority of the JN's and then moves out. i.e succeeds 
and returns if it gets response from 2 JN. The thread for the third JN keeps on 
trying 10 times on ConnectException. But since the tailing period is quite low, 
By the time one stuck thread completes. Similar retrying stuck threads, Piles 
up. Leading to OOM

> SBN : 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
>Priority: Major
>
> {noformat}
> 2019-07-04 17:35:54,064 | INFO  | Logger channel (from parallel executor) to 
> XXX/XXX | Retrying connect to server: XXX/XXX. 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
(v7.6.14#76016)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org