[
https://issues.apache.org/jira/browse/HDFS-14648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16915262#comment-16915262
]
He Xiaoqiao commented on HDFS-14648:
------------------------------------
Thanks [~leosun08] for your contributions. I try to learn
[^HDFS-14648.004.patch] with some minor comments,
1. Some codestyles reported by Jenkins above, Please take a look.
2. It is better to locate same module configuration together in
hdfs-default.xml to find conveniently relevant items, just suggest to define
`dfs.client.deadnode.detect.enabled` together with other `dfs.client.*` items.
3. Some method name is open to different interpretations. for instance,
`DeadNodeDetector#removeNodeFromDetect`, my first impression is that remove
node from `deadNodes` shared by all DFSInputStreams in the same DFSClient,
however it just mean remove this node from `localNodes` which just visible to
some one DFSInputStream actually. right?
4. about DeadNodeDetector, I do not get why set different STATEs for
`DeadNodeDetector`, will be used by the following implements?
5. about DeadNodeDetector#run, does it need to catch InterruptedException out
of while loop and return?
6. Some constant such as '5000'/'10000' but not explain why, I think we should
define this constant at the begin of Class and add some annotation.
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`.
Thanks [~leosun08] 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
>
>
> 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]