[
https://issues.apache.org/jira/browse/HDFS-10719?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17568596#comment-17568596
]
Jan Van Besien edited comment on HDFS-10719 at 7/25/22 8:06 AM:
----------------------------------------------------------------
This ticket was closed as a duplicate of HDFS-4210 but I think that is a
mistake and the problem still exists.
HDFS-4210 is about throwing an UnknownHostException rather than a
NullPointerException. In HDFS-4210 the context of the exception is namenode
formatting (hence FSEditLog.initJournalsForWrite). It seems sensible to only
allow that when all journal nodes are indeed available.
This ticket however, is about starting namenodes (that are already formatted,
hence FSEditLog.initSharedJournalsForRead). It seems that namenodes should be
able to start as long as there is a majority of journalnodes? That's what the
patches on this ticket try to fix.
It might not be so common that a journal node is unavailable because its DNS
address cannot be resolved. But it can happen. The first 3 patches try to count
the number of resolved journal node addresses. The latest patch doesn't do
that, but simply retains the ones that are resolved to then rely on
org.apache.hadoop.hdfs.qjournal.client.AsyncLoggerSet#getMajoritySize further
on, I assume. I don't know the HDFS codebase very well, but that sounds like a
plausible approach to me.
I ran into this problem in HDFS 3.3.1.
Edit: this is HDFS-4957.
was (Author: janvanbesien):
This ticket was closed as a duplicate of HDFS-4210 but I think that is a
mistake and the problem still exists.
HDFS-4210 is about throwing an UnknownHostException rather than a
NullPointerException. In HDFS-4210 the context of the exception is namenode
formatting (hence FSEditLog.initJournalsForWrite). It seems sensible to only
allow that when all journal nodes are indeed available.
This ticket however, is about starting namenodes (that are already formatted,
hence FSEditLog.initSharedJournalsForRead). It seems that namenodes should be
able to start as long as there is a majority of journalnodes? That's what the
patches on this ticket try to fix.
It might not be so common that a journal node is unavailable because its DNS
address cannot be resolved. But it can happen. The first 3 patches try to count
the number of resolved journal node addresses. The latest patch doesn't do
that, but simply retains the ones that are resolved to then rely on
org.apache.hadoop.hdfs.qjournal.client.AsyncLoggerSet#getMajoritySize further
on, I assume. I don't know the HDFS codebase very well, but that sounds like a
plausible approach to me.
I ran into this problem in HDFS 3.3.1.
> In HA, Namenode is failed to start If any of the Quorum hostname is
> unresolved.
> -------------------------------------------------------------------------------
>
> Key: HDFS-10719
> URL: https://issues.apache.org/jira/browse/HDFS-10719
> Project: Hadoop HDFS
> Issue Type: Bug
> Components: journal-node, namenode
> Affects Versions: 2.7.1
> Environment: HDP-2.4
> Reporter: Karthik Palanisamy
> Assignee: Karthik Palanisamy
> Priority: Major
> Labels: patch
> Attachments: HDFS-10719-1.patch, HDFS-10719-2.patch,
> HDFS-10719-3.patch, HDFS-10719-4.patch
>
>
> 2016-08-03 02:53:53,760 ERROR namenode.NameNode (NameNode.java:main(1712)) -
> Failed to start namenode.
> java.lang.IllegalArgumentException: Unable to construct journal,
> qjournal://xxxx1:8485;xxxx2:8485;xxxx3:8485/shva
> at
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.createJournal(FSEditLog.java:1637)
> at
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.initJournals(FSEditLog.java:282)
> at
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.initSharedJournalsForRead(FSEditLog.java:260)
> at
> org.apache.hadoop.hdfs.server.namenode.FSImage.initEditLog(FSImage.java:789)
> at
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:634)
> at
> org.apache.hadoop.hdfs.server.namenode.FSImage.recoverTransitionRead(FSImage.java:294)
> at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFSImage(FSNamesystem.java:983)
> at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFromDisk(FSNamesystem.java:688)
> at
> org.apache.hadoop.hdfs.server.namenode.NameNode.loadNamesystem(NameNode.java:662)
> at
> org.apache.hadoop.hdfs.server.namenode.NameNode.initialize(NameNode.java:726)
> at
> org.apache.hadoop.hdfs.server.namenode.NameNode.<init>(NameNode.java:951)
> at
> org.apache.hadoop.hdfs.server.namenode.NameNode.<init>(NameNode.java:935)
> at
> org.apache.hadoop.hdfs.server.namenode.NameNode.createNameNode(NameNode.java:1641)
> at
> org.apache.hadoop.hdfs.server.namenode.NameNode.main(NameNode.java:1707)
> Caused by: java.lang.reflect.InvocationTargetException
> at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> Method)
> at
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> at
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> at java.lang.reflect.Constructor.newInstance(Constructor.java:422)
> at
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.createJournal(FSEditLog.java:1635)
> ... 13 more
> Caused by: java.lang.NullPointerException
> at
> org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannelMetrics.getName(IPCLoggerChannelMetrics.java:107)
> at
> org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannelMetrics.create(IPCLoggerChannelMetrics.java:91)
> at
> org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel.<init>(IPCLoggerChannel.java:178)
> at
> org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel$1.createLogger(IPCLoggerChannel.java:156)
> at
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager.createLoggers(QuorumJournalManager.java:367)
> at
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager.createLoggers(QuorumJournalManager.java:149)
> at
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager.<init>(QuorumJournalManager.java:116)
> at
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager.<init>(QuorumJournalManager.java:105)
> ... 18 more
> 2016-08-03 02:53:53,765 INFO util.ExitUtil (ExitUtil.java:terminate(124)) -
> Exiting with status 1
> 2016-08-03 02:53:53,768 INFO namenode.NameNode (LogAdapter.java:info(47)) -
> SHUTDOWN_MSG:
> *and the failover is not successful*
> I have attached the patch, It allows the Namenode to start if the majority of
> the Quorums are resolvable.
> throws warning if the quorum is unresolvable.
> throws Unknown host exception if the majority of the journals are
> unresolvable.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]