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

stack commented on HDFS-5776:
-----------------------------

Nice numbers [~xieliang007]

When you get a chance, there were a few questions in the previous review notes.

Nit: This seems extraneous in the new Callable:

+        return null;

Nit: No need of the intermediary 'instance' assignment -- just assign to 
'injector'?

+    // Set up the InjectionHandler
+    DFSClientFaultInjector.instance = Mockito
+        .mock(DFSClientFaultInjector.class);
+    DFSClientFaultInjector injector = DFSClientFaultInjector.instance;

Nit: Should the '60' below here:

+        Thread.sleep(60);

Be more related to the the '100' you pass as the 
DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS config?  Be half of whatever 
DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS is?   My concern is that someone 
could change one of these settings not realizing they are related (they are, 
right)?

Is it possible that on a slow machine -- such as apache jenkins -- that we may 
get a hedged read when we do not expect it ?

+      // assert that there were no hedged reads. 60ms + delta < 100ms

i.e. could this test turn flakey on a strained testing infrastructure?

Shut down this executor in the finally?  Don't want it sticking around.

+      ExecutorService executor = Executors.newFixedThreadPool(numHedgedReads);

For sure this will always trigger though the number of futures == 
numHedgedReads (should futures == numHedgedReads + 1 to be sure?)

+      assertTrue(metrics.getHedgedReadOpsInCurThread() > 0);

Nice test.

Your nice new metrics showed in your above test?  They made sense (I suppose 
they must basically work since your test relies on them).

You need the below new log?

+          DFSClient.LOG.warn("Could not obtain block " + block + errMsg
+              + ". Throw a BlockMissingException");

s/Throw/Throwing/

Make the above log message match the content of the BlockMissingException so it 
easier connecting the two emissions (Later in the patch you actually do this).

Needs a space between ie and errMsg? 

+            + " from any node: " + ie + errMsg

When we get to the end of the pipeline here; i.e. all datanodes have been 
tried, what happens?

{code}
+    while (true) {
+      DNAddrPair retval = chooseDataNode(block);
+      try {
+        actualGetFromOneDataNode(retval, block, start, end, buf, offset,
+            corruptedBlockMap);
+        return;
+      } catch (IOException e) {
+        // Ignore. Already processed inside the function.
+        // Loop through to try the next node.
+      }
+    }
{code}

Seems like the above is a common idiom in DFSClient. 

Say why it is ok to ignore the IOE at this point in the comment.

+            // ignore fetchBlockAt IOException

This is good:

-            DFSClient.LOG.debug("Connection failure ", e);
+            DFSClient.LOG.debug("Connection failure: " + msg, e);

I suppose moving this into finally would be messier than what you have done 
where you add it to the end of the if and the else clauses when exception:

-      // Put chosen node into dead list, continue
-      addToDeadNodes(chosenNode);

Should fetchBlockByteRangeSpeculative be called fetchBlockByteRangeHedge or 
hedgedFetchBlockByteRange.... 'hedged' fetches is what this patch introduces.  
'speculative' may confuse.  At least add a comment that the method is about 
'hedged' fetches.

So on a dfsclient instance, we can flip hedged reads on and off?
+  public void enableHedgedReads() {
+    allowHedgedReads = true;
+  }

ThreadPoolExecutor should make daemon threads?

Is this a good idea?

getHedgedReadsThreadPool

Should be kept internal to DFSClient.

Patch is great [~xieliang007]









> Support 'hedged' reads in DFSClient
> -----------------------------------
>
>                 Key: HDFS-5776
>                 URL: https://issues.apache.org/jira/browse/HDFS-5776
>             Project: Hadoop HDFS
>          Issue Type: Improvement
>          Components: hdfs-client
>    Affects Versions: 3.0.0
>            Reporter: Liang Xie
>            Assignee: Liang Xie
>         Attachments: HDFS-5776-v2.txt, HDFS-5776-v3.txt, HDFS-5776.txt
>
>
> This is a placeholder of hdfs related stuff backport from 
> https://issues.apache.org/jira/browse/HBASE-7509
> The quorum read ability should be helpful especially to optimize read outliers
> we can utilize "dfs.dfsclient.quorum.read.threshold.millis" & 
> "dfs.dfsclient.quorum.read.threadpool.size" to enable/disable the hedged read 
> ability from client side(e.g. HBase), and by using DFSQuorumReadMetrics, we 
> could export the interested metric valus into client system(e.g. HBase's 
> regionserver metric).
> The core logic is in pread code path, we decide to goto the original 
> fetchBlockByteRange or the new introduced fetchBlockByteRangeSpeculative per 
> the above config items.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to