[
https://issues.apache.org/jira/browse/HDFS-14648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16918707#comment-16918707
]
Lisheng Sun commented on HDFS-14648:
------------------------------------
Thank [~hexiaoqiao] for deep reivew and good suggestion.
{quote}
4. about DeadNodeDetector, I do not get why set different STATEs for
`DeadNodeDetector`, will be used by the following implements?
{quote}
That set different STATEs for `DeadNodeDetector` is when DeadNodeDetector is
in different state, it does different in future implementations. E.g when
DeadNodeDetector is CHECK_DEAD state, it can detect dead node in
DeadNodeDetector#deadnode.
It is also eaiser for future expansion.
{quote}
7. IIUC, some nodes are detected as dead, it should not be in next pipeline,
right? but I do not see anywhere to add this deadNodes to `excludeNodes`
{quote}
If datanode is consider as deadnode, it is placed in DFSInputsteam#deadnode and
DeadNodeDetector#deadnode. Other dfsintputSream should no longer access this
datanode.
Updatd this patch and uploaded the v005 patch. Could you mind continue to take
a review for this patch? Thank [~hexiaoqiao] again.
> DeadNodeDetector basic model
> ----------------------------
>
> Key: HDFS-14648
> URL: https://issues.apache.org/jira/browse/HDFS-14648
> Project: Hadoop HDFS
> Issue Type: Sub-task
> Reporter: Lisheng Sun
> Assignee: Lisheng Sun
> Priority: Major
> Attachments: HDFS-14648.001.patch, HDFS-14648.002.patch,
> HDFS-14648.003.patch, HDFS-14648.004.patch, HDFS-14648.005.patch
>
>
> This Jira constructs DeadNodeDetector state machine model. The function it
> implements as follow:
> # When a DFSInputstream is opened, a BlockReader is opened. If some DataNode
> of the block is found to inaccessible, put the DataNode into
> DeadNodeDetector#deadnode.(HDFS-14649) will optimize this part. Because when
> DataNode is not accessible, it is likely that the replica has been removed
> from the DataNode.Therefore, it needs to be confirmed by re-probing and
> requires a higher priority processing.
> # DeadNodeDetector will periodically detect the Node in
> DeadNodeDetector#deadnode, If the access is successful, the Node will be
> moved from DeadNodeDetector#deadnode. Continuous detection of the dead node
> is necessary. The DataNode need rejoin the cluster due to a service
> restart/machine repair. The DataNode may be permanently excluded if there is
> no added probe mechanism.
> # DeadNodeDetector#dfsInputStreamNodes Record the DFSInputstream using
> DataNode. When the DFSInputstream is closed, it will be moved from
> DeadNodeDetector#dfsInputStreamNodes.
> # Every time get the global deanode, update the DeadNodeDetector#deadnode.
> The new DeadNodeDetector#deadnode Equals to the intersection of the old
> DeadNodeDetector#deadnode and the Datanodes are by
> DeadNodeDetector#dfsInputStreamNodes.
> # DeadNodeDetector has a switch that is turned off by default. When it is
> closed, each DFSInputstream still uses its own local deadnode.
> # This feature has been used in the XIAOMI production environment for a long
> time. Reduced hbase read stuck, due to node hangs.
> # Just open the DeadNodeDetector switch and you can use it directly. No
> other restrictions. Don't want to use DeadNodeDetector, just close it.
> {code:java}
> if (sharedDeadNodesEnabled && deadNodeDetector == null) {
> deadNodeDetector = new DeadNodeDetector(name);
> deadNodeDetectorThr = new Daemon(deadNodeDetector);
> deadNodeDetectorThr.start();
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.3.2#803003)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]