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

Yiqun Lin commented on HDFS-14648:
----------------------------------

Thanks [~leosun08] for updating the patch! It looks more readable now. Some 
deep review comments from me:

*ClientContext.java*
1.Can we use {{deadNodeDetectionEnabled}} to replace 
{{sharedDeadNodesEnabled}}? This name will keep consistent with 
DeadNodeDetector. Can you replace this in the whole patch, including in some 
method comments?

*DFSClient.java*
1. Why we add the deadnode from dfsInputstream again? Just to get the latest 
dead node that hasn't  been detected? DeadNodeDetector should already add the  
dfsInputstream dead node when the deadnode was detected.
{code}
  public ConcurrentHashMap<DatanodeInfo, DatanodeInfo> getDeadNodes(
+      DFSInputStream dfsInputStream) {
+    if (clientContext.isSharedDeadNodesEnabled()) {
+      ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
+          new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
+      if (dfsInputStream != null) {
+        deadNodes.putAll(dfsInputStream.getLocalDeadNodes());
+      }
+
+      Set<DatanodeInfo> detectDeadNodes =
+          clientContext.getDeadNodeDetector().getDeadNodesToDetect();
+      for (DatanodeInfo detectDeadNode : detectDeadNodes) {
+        deadNodes.put(detectDeadNode, detectDeadNode);
+      }
...
{code}
2. Can we redundant '{}'?
{code}
LOG.debug("DeadNode detection is not enabled or given block {} is null, " +
+              "skip to remove node {}.", locatedBlocks);
{code} 
to
{code}
LOG.debug("DeadNode detection is not enabled or given block {} is null, " +
+              "skip to remove node.", locatedBlocks);
{code} 
 
*DeadNodeDetector.java*
1. Can we comment the name as Client context name
{code}
+  /**
+   * Client context name.
+   */
+  private String name;
{code}
2.Can we use datanodeuuid get from DatanodeInfo as the key? We use the same 
type object here and looks confused. datanodeuuid is the identity of one DN 
node.
{code}
private final ConcurrentHashMap<String, DatanodeInfo> deadNodes;
{code}
3. I think it will be better to print out the detected dead node info here.
{code}
LOG.debug("Current detector state {}, the detected nodes: {}.",....);
{code}
4. Two comments for this:
1) Update the method name to clearAndGetDetectedDeadNodes
2) The line newDeadNodes.retainAll(deadNodes.values()); should not be correct, 
it will let newDeadNodes be same with old deadnodes.
{code}
+  public synchronized Set<DatanodeInfo> getDeadNodesToDetect() {
+    // remove the dead nodes who doesn't have any inputstream first
+    Set<DatanodeInfo> newDeadNodes = new HashSet<DatanodeInfo>();
+    for (HashSet<DatanodeInfo> datanodeInfos : dfsInputStreamNodes.values()) {
+      newDeadNodes.addAll(datanodeInfos);
+    }
+
+    newDeadNodes.retainAll(deadNodes.values());
+
+    for (DatanodeInfo datanodeInfo : deadNodes.values()) {
+      if (!newDeadNodes.contains(datanodeInfo)) {
+        deadNodes.remove(datanodeInfo);
+      }
+    }
+    return newDeadNodes;
+  }
{code}

*TestDFSClientDetectDeadNodes.java*
Comments for testDetectDeadNodeInBackground  and 
testDeadNodeMultipleDFSInputStream: 
1). Can we check the deadnode size again after closing the dfs input stream?
2). Can we use DeadDetector#getDeadNodesToDetect to check dead node as well?
{code}
FSDataInputStream in = fs.open(filePath);
+    try {
+      try {
+        in.read();
+      } catch (BlockMissingException e) {
+      }
+
+      DFSInputStream din = (DFSInputStream) in.getWrappedStream();
+      assertEquals(3, din.getDFSClient().getDeadNodes(din).size());
       // use DeadDetector to get dead node as well
+    } finally {
+      in.close();
+      fs.delete(new Path("/testDetectDeadNodeInBackground"),
+              true);
      // check the dead node again here, the dead node is expected be removed
+    }
{code}
3. Can we check the dead node detail info as well, like dn uuid?
{code}
DFSInputStream din2 = (DFSInputStream) in1.getWrappedStream();
+      assertEquals(1, din1.getDFSClient().getDeadNodes(din1).size());
+      assertEquals(1, din2.getDFSClient().getDeadNodes(din2).size());
//  check the dn uuid of dead node to see if its expected dead node
{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
>
>
> 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

Reply via email to