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

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

The patch overall looks good, but some places seem not readable. I'd like to 
give some minor comments to improve this firstly. Will give further review 
comments since I am still in reviewing.

*ClientContext.java*
 1.'Detect the dead datanodes n advance', should be 'Detect the dead datanodes 
in advance,'
 2.Can we add the comment for these two methods since they frequently be called 
in other places?

*DFSClient.java*
 1.Simplified the comment for method ConcurrentHashMap<DatanodeInfo, 
DatanodeInfo> getDeadNodes(DFSInputStream dfsInputStream)
{noformat}
+  /**
+   * If sharedDeadNodesEnabled is true, return the dead nodes are detected by
+   * all the DFSInputStreams in the same client. Otherwise return the dead 
nodes
+   * are detected by this DFSInputStream.
+   */
{noformat}
to
{noformat}
+  /**
+   * If sharedDeadNodesEnabled is true, return the dead nodes that detected by
+   * all the DFSInputStreams in the same client. Otherwise return the dead 
nodes
+   * that detected by given DFSInputStream.
+   */
{noformat}
2.Simplified the comment for method isDeadNode(DFSInputStream dfsInputStream, 
DatanodeInfo datanodeInfo)
{noformat}
+  /**
+   * If sharedDeadNodesEnabled is true, judgement based on whether this 
datanode
+   * is included or not in DeadNodeDetector#deadnodes. Otherwise judgment based
+   * on whether it is included or not in DFSInputStream#deadnodes.
+   */
{noformat}
to
{noformat}
+  /**
+   * If sharedDeadNodesEnabled is true, judgement based on whether this 
datanode
+   * is included or not in DeadNodeDetector. Otherwise judgment based given
+   * DFSInputStream.
+   */
{noformat}
3. It will be better to add one additional log here and update the method name.
{code:java}
 
+  /**
+   * Add given datanode in DeadNodeDetector.
+   */
public void addNodeToDeadNodeDetector(DFSInputStream dfsInputStream,
+      DatanodeInfo datanodeInfo) {
+    if (!isSharedDeadNodesEnabled()) {
        LOG.debug("DeadNode detection is not enabled, skip to add node {}.", 
datanodeInfo);
+      return;
+    }
+    clientContext.getDeadNodeDetector().addNodeToDetect(dfsInputStream,
+        datanodeInfo);
+  }
{code}
4. The similar change for method removeNodeFromDetectByDFSInputStream
{code:java}
+  /**
+   * Remove given datanode from DeadNodeDetector.
+   */
+  public void removeNodeFromDeadNodeDetector(
+      DFSInputStream dfsInputStream, DatanodeInfo datanodeInfo) {
+    if (!isSharedDeadNodesEnabled()) {
        LOG.debug("DeadNode detection is not enabled, skip to remove node {}.", 
datanodeInfo);
+      return;
+    }
+    clientContext.getDeadNodeDetector()
+        .removeNodeFromDetectByDFSInputStream(dfsInputStream, datanodeInfo);
+  }
{code}
5. Update for removeNodeFromDeadNodeDetector
{code:java}
   /**
+   * Remove datanodes that given block placed on from DeadNodeDetector.
+   */
+  public void removeNodeFromDetectByDFSInputStream(
+      DFSInputStream dfsInputStream, LocatedBlocks locatedBlocks) {
+    if (!isSharedDeadNodesEnabled() || locatedBlocks == null) {
        LOG.debug("DeadNode detection is not enabled or given block is null, 
skip to remove node.",);
+      return;
+    }
+    for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
+      for (DatanodeInfo datanodeInfo : locatedBlock.getLocations()) {
+        removeNodeFromDetectByDFSInputStream(dfsInputStream, datanodeInfo);
+      }
+    }
+  }
{code}
 * DFSInputStream.java*
 1. Update method name getDfsClient to getDFSClient
{code:java}
+  public DFSClient getDFSClient() {
+    return dfsClient;
+  } 
{code}
2. Can we reduce the visibility of these methods? I don't think all of them 
need to be public, protected or private should be enough.
{code:java}
+  public void removeFromLocalDeadNodes(DatanodeInfo dnInfo) {
+    deadNodes.remove(dnInfo);
+  }
+
+  public ConcurrentHashMap<DatanodeInfo, DatanodeInfo> getLocalDeadNodes() {
+    return deadNodes;
+  }
+
+  public void clearLocalDeadNodes() {
+    deadNodes.clear();
+  }
+
+  public DFSClient getDfsClient() {
+    return dfsClient;
+  }
{code}

 * DeadNodeDetector.java*
 1. Can we add the comment for the sleep time and name?
{code:java}
+  private static final long ERROR_SLEEP_MS = 5000;
+  private static final long IDLE_SLEEP_MS = 10000;
+
+  private String name;
{code}
2. Update the log to use the parameter way and 'start' should be 'Start'.
{code:java}
LOG.info("start dead node detector for DFSClient " + this.name);
{code}
to
{code:java}
LOG.info("Start dead node detector for DFSClient {}.", name);
{code}
3.Update the state log
{code:java}
LOG.debug("state " + state);
{code}
to
{code:java}
LOG.debug("Current detector state {}.", state);
{code}

 * StripeReader.java*
 The class doesn't need to do any change since we don't make name change for 
method getDFSClient.

 * HdfsClientConfigKeys.java*
 I prefer to define the setting name to 
{{dfs.client.deadnode.detection.enabled}}

*hdfs-default.xml*
 Update the description to
{noformat}
Set to true to enable dead node detection in client side. Then all the 
DFSInputStreams of the same client can share the dead node information.
{noformat}
*TestDFSClientDetectDeadNodes.java*
 1. Do we need so long timeout value for the ut?
{code:java}
@Test(timeout = 60000000)
+  public void testDetectDeadNodeInBackground() throws IOException {
{code}
2. Update the asset judgement
{code:java}
assertTrue(din.getDfsClient().getDeadNodes(din).size() == 3);
{code}
to
{code:java}
assertEquals(3, din.getDfsClient().getDeadNodes(din).size());
{code}
3. Can we clear the test file after the verification?
 4. Can we use a different path in {{testDeadNodeMultipleDFSInputStream}}?
{code:java}
Path filePath = new Path("/testNodeBecomeDead");
{code}
5. The same suggestion for following lines.
{code:java}
assertTrue(din1.getDfsClient().getDeadNodes(din1).size() == 1);
assertTrue(din2.getDfsClient().getDeadNodes(din2).size() == 1);
{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
>
>
> 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: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to