[ https://issues.apache.org/jira/browse/HDFS-14648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16972571#comment-16972571 ]
Yiqun Lin commented on HDFS-14648: ---------------------------------- Thanks [~leosun08] , the patch almost looks good now, only some minor comments: *DFSInputStream.java* 1. There are one additional places we can add the addNodeToDeadNodeDetector call in {{createBlockReader}} {code:java} boolean createBlockReader(LocatedBlock block, long offsetInBlock, LocatedBlock[] targetBlocks, BlockReaderInfo[] readerInfos, ... } else { //TODO: handles connection issues DFSClient.LOG.warn("Failed to connect to " + dnInfo.addr + " for " + "block" + block.getBlock(), e); // re-fetch the block in case the block has been moved fetchBlockAt(block.getStartOffset()); addToLocalDeadNodes(dnInfo.info); // <---- } } {code} *DeadNodeDetector.java* 1.Can you address this comment that missed? {quote}1. Can we comment the name as Client context name + /** + * Client context name. + */ + private String name; {quote} 2. We can use the containsKey to check {code:java} public boolean isDeadNode(DatanodeInfo datanodeInfo) { return deadNodes.containsKey((datanodeInfo.getDatanodeUuid()); } {code} Also we can use the key to remove in method clearAndGetDetectedDeadNodes {code:java} for (DatanodeInfo datanodeInfo : deadNodes.values()) { if (!newDeadNodes.contains(datanodeInfo)) { deadNodes.remove(datanodeInfo.getDatanodeUuid()); } } {code} 3. We can periodically call clearAndGetDetectedDeadNodes to make deadNodes list be refreshed. I think deadNodes list can be a little staled when the local dead node is cleared in dfs input stream. {code:java} public void run() { while (true) { clearAndGetDetectedDeadNodes(); LOG.debug("Current detector state {}, the detected nodes: {}.", state, deadNodes.values()); switch (state) { {code} 4. Not fully get this. Why we still call this in the latest patch? Can you explain for this? {noformat} newDeadNodes.retainAll(deadNodes.values()); {noformat} *TestDFSClientDetectDeadNodes.java* 1. Can you rename the unit test name from {{TestDFSClientDetectDeadNodes}} to {{TestDeadNodeDetection}}? And simplified the comment to this: {noformat} +/** + * Tests for dead node detection in DFSClient. + */ +public class TestDeadNodeDetection { {noformat} Two other name updated: * testDetectDeadNodeInBackground --> testDeadNodeDetectionInBackground * testDeadNodeMultipleDFSInputStream --> testDeadNodeDetectionInMultipleDFSInputStream 2. No needed to call {{ThreadUtil.sleepAtLeastIgnoreInterrupts(10 * 1000L);}} I think. 3. Can we extract the DFSClient here? I see we call many times getDFSClient(). {code:java} assertEquals(1, din1.getDFSClient().getDeadNodes(din1).size()); assertEquals(1, din1.getDFSClient().getClientContext() {code} > 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, > HDFS-14648.006.patch, HDFS-14648.007.patch, HDFS-14648.008.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.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org