Details

    • Type: Improvement
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 3.0.0-alpha1
    • Fix Version/s: 2.4.0
    • Component/s: hdfs-client
    • Labels:
      None
    • Hadoop Flags:
      Reviewed
    • Release Note:
      Hide
      If a read from a block is slow, start up another parallel, 'hedged' read against a different block replica. We then take the result of which ever read returns first (the outstanding read is cancelled). This 'hedged' read feature will help rein in the outliers, the odd read that takes a long time because it hit a bad patch on the disc, etc.

      This feature is off by default. To enable this feature, set <code>dfs.client.hedged.read.threadpool.size</code> to a positive number. The threadpool size is how many threads to dedicate to the running of these 'hedged', concurrent reads in your client.

      Then set <code>dfs.client.hedged.read.threshold.millis</code> to the number of milliseconds to wait before starting up a 'hedged' read. For example, if you set this property to 10, then if a read has not returned within 10 milliseconds, we will start up a new read against a different block replica.

      This feature emits new metrics:

      + hedgedReadOps
      + hedgeReadOpsWin -- how many times the hedged read 'beat' the original read
      + hedgedReadOpsInCurThread -- how many times we went to do a hedged read but we had to run it in the current thread because dfs.client.hedged.read.threadpool.size was at a maximum.
      Show
      If a read from a block is slow, start up another parallel, 'hedged' read against a different block replica. We then take the result of which ever read returns first (the outstanding read is cancelled). This 'hedged' read feature will help rein in the outliers, the odd read that takes a long time because it hit a bad patch on the disc, etc. This feature is off by default. To enable this feature, set <code>dfs.client.hedged.read.threadpool.size</code> to a positive number. The threadpool size is how many threads to dedicate to the running of these 'hedged', concurrent reads in your client. Then set <code>dfs.client.hedged.read.threshold.millis</code> to the number of milliseconds to wait before starting up a 'hedged' read. For example, if you set this property to 10, then if a read has not returned within 10 milliseconds, we will start up a new read against a different block replica. This feature emits new metrics: + hedgedReadOps + hedgeReadOpsWin -- how many times the hedged read 'beat' the original read + hedgedReadOpsInCurThread -- how many times we went to do a hedged read but we had to run it in the current thread because dfs.client.hedged.read.threadpool.size was at a maximum.

      Description

      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.

      1. HDFS-5776.txt
        29 kB
        Liang Xie
      2. HDFS-5776-v10.txt
        31 kB
        Liang Xie
      3. HDFS-5776-v11.txt
        31 kB
        Liang Xie
      4. HDFS-5776-v12.txt
        31 kB
        stack
      5. HDFS-5776-v12.txt
        31 kB
        stack
      6. HDFS-5776-v13.wip.txt
        31 kB
        stack
      7. HDFS-5776-v14.txt
        30 kB
        stack
      8. HDFS-5776-v15.txt
        32 kB
        stack
      9. HDFS-5776-v17.txt
        33 kB
        stack
      10. HDFS-5776-v17.txt
        33 kB
        stack
      11. HDFS-5776v18.txt
        34 kB
        stack
      12. HDFS-5776-v2.txt
        29 kB
        Liang Xie
      13. HDFS-5776v21.txt
        35 kB
        stack
      14. HDFS-5776v21-branch2.txt
        34 kB
        stack
      15. HDFS-5776-v3.txt
        30 kB
        Liang Xie
      16. HDFS-5776-v4.txt
        30 kB
        Liang Xie
      17. HDFS-5776-v5.txt
        29 kB
        Liang Xie
      18. HDFS-5776-v6.txt
        30 kB
        Liang Xie
      19. HDFS-5776-v7.txt
        29 kB
        Liang Xie
      20. HDFS-5776-v8.txt
        30 kB
        Liang Xie
      21. HDFS-5776-v9.txt
        31 kB
        Liang Xie

        Issue Links

          Activity

          Hide
          xieliang007 Liang Xie added a comment -

          I have made a raw patch against 2.0 branch yesterday, will upload the patch once testing done.

          Show
          xieliang007 Liang Xie added a comment - I have made a raw patch against 2.0 branch yesterday, will upload the patch once testing done.
          Hide
          sureshms Suresh Srinivas added a comment -

          Is this really quorum read or just reading in parallel? Can you please add more details to the description?

          Show
          sureshms Suresh Srinivas added a comment - Is this really quorum read or just reading in parallel? Can you please add more details to the description?
          Hide
          stack stack added a comment -

          Good on you Liang Xie Is it // read or start up read on second replica if first is slow? Thanks for working on this.

          Show
          stack stack added a comment - Good on you Liang Xie Is it // read or start up read on second replica if first is slow? Thanks for working on this.
          Hide
          xieliang007 Liang Xie added a comment -

          thanks for comments, probably "quorum read" is not accurate, just copy from fb-20 & 0.89-fb branches' naming.
          The core logic is in pread op, send a callable read request against dn1, if no response until a timeout reach, then chooseDN will pick another DN to send the similar read request.

          This idea is similar with Jeff Dean's : <The tail at scale-dean>, in that article, Jeff named a "Hedged requests", and a clever defer sending request optimization to limit the additional load.

          Show
          xieliang007 Liang Xie added a comment - thanks for comments, probably "quorum read" is not accurate, just copy from fb-20 & 0.89-fb branches' naming. The core logic is in pread op, send a callable read request against dn1, if no response until a timeout reach, then chooseDN will pick another DN to send the similar read request. This idea is similar with Jeff Dean's : <The tail at scale-dean>, in that article, Jeff named a "Hedged requests", and a clever defer sending request optimization to limit the additional load.
          Hide
          xieliang007 Liang Xie added a comment -

          Please feel free to modify this JIRA's title and description, i'm not a native speaker, probably not good at the detailed semantics

          Show
          xieliang007 Liang Xie added a comment - Please feel free to modify this JIRA's title and description, i'm not a native speaker, probably not good at the detailed semantics
          Hide
          stack stack added a comment -

          I had a go at the subject for you Liang Xie I am a native speaker but when others read my writing or hear me talk, they wonder.

          Show
          stack stack added a comment - I had a go at the subject for you Liang Xie I am a native speaker but when others read my writing or hear me talk, they wonder.
          Hide
          xieliang007 Liang Xie added a comment -

          let's see what QA robot will say

          Show
          xieliang007 Liang Xie added a comment - let's see what QA robot will say
          Hide
          xieliang007 Liang Xie added a comment -

          "mvn clean test -Dtest=TestPread" passed locally

          Show
          xieliang007 Liang Xie added a comment - "mvn clean test -Dtest=TestPread" passed locally
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12623328/HDFS-5776.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          -1 javac. The applied patch generated 1546 javac compiler warnings (more than the trunk's current 1545 warnings).

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The following test timeouts occurred in hadoop-hdfs-project/hadoop-hdfs:

          org.apache.hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5897//testReport/
          Javac warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5897//artifact/trunk/patchprocess/diffJavacWarnings.txt
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5897//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12623328/HDFS-5776.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. -1 javac . The applied patch generated 1546 javac compiler warnings (more than the trunk's current 1545 warnings). +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The following test timeouts occurred in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5897//testReport/ Javac warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5897//artifact/trunk/patchprocess/diffJavacWarnings.txt Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5897//console This message is automatically generated.
          Hide
          xieliang007 Liang Xie added a comment -

          v2 should address the javadoc and failed case

          Show
          xieliang007 Liang Xie added a comment - v2 should address the javadoc and failed case
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12623601/HDFS-5776-v2.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5910//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5910//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12623601/HDFS-5776-v2.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5910//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5910//console This message is automatically generated.
          Hide
          xieliang007 Liang Xie added a comment -

          all are green now, nice

          Show
          xieliang007 Liang Xie added a comment - all are green now, nice
          Hide
          stack stack added a comment -

          Liang Xie

          Nit: The 'return null' necessary in below given it Void return type:

          + public Void call() throws IOException

          { + pReadFile(fileSys, file); + return null; + }

          We should rename these methods given this is not 'quorum' reading:

          testQuorumPreadDFSBasic
          

          ... and here testMaxOutQuorumPool

          And this variable name needs changing? numQuorumPoolThreads... and here DFS_DFSCLIENT_QUORUM_READ_THREADPOOL_SIZE

          ... more review to come (have to head out)

          Show
          stack stack added a comment - Liang Xie Nit: The 'return null' necessary in below given it Void return type: + public Void call() throws IOException { + pReadFile(fileSys, file); + return null; + } We should rename these methods given this is not 'quorum' reading: testQuorumPreadDFSBasic ... and here testMaxOutQuorumPool And this variable name needs changing? numQuorumPoolThreads... and here DFS_DFSCLIENT_QUORUM_READ_THREADPOOL_SIZE ... more review to come (have to head out)
          Hide
          stack stack added a comment -

          This looks like copy paste issue:

          + // assert that there were no quorum reads. 60ms + delta < 100ms
          + assertTrue(metrics.getParallelReadOps() > 0);

          Here you are asserting that there IS // 'hedged' reads going on.

          I was wondering if 'hedged requests' a good name for this feature and going by the definition from your citation, it is good by me:

          Hedged requests. A simple way to curb latency variability is to issue the same request to multiple replicas and use the results from whichever replica responds first. We term such requests "hedged requests" because a client first sends one request to the replica believed to be the most appropriate, but then falls back on sending a secondary request after some brief delay. The client cancels remaining outstanding requests once the first result is received. Although naive implementations of this technique typically add unacceptable additional load, many variations exist that give most of the latency-reduction effects while increasing load only modestly.

          One such approach is to defer sending a secondary request until the first request has been outstanding for more than the 95th-percentile expected latency for this class of requests. This approach limits the additional load to approximately 5% while substantially shortening the latency tail. ....

          http://cacm.acm.org/magazines/2013/2/160173-the-tail-at-scale/fulltext

          We'd change this to be allowHedgeReads?

          + public volatile boolean allowParallelReads = false;

          This would be hedgedReadThresholdMillis

          + private volatile long quorumReadThresholdMillis;

          ... and so on.

          Or do you see parallel reads as different from hedged reads? ('Tied requests' from your citation).

          These are public so clients like hbase can tinker with them:

          +
          +  public void setQuorumReadTimeout(long timeoutMillis) {
          +    this.quorumReadThresholdMillis = timeoutMillis;
          +  }
          +
          +  public long getQuorumReadTimeout() {
          +    return this.quorumReadThresholdMillis;
          +  }
          +
          +  public void enableParallelReads() {
          +    allowParallelReads = true;
          +  }
          +
          +  public void disableParallelReads() {
          +    allowParallelReads = false;
          +  }
          

          So if the requests are > the configured number, we run in the current thread which is better than rejecting the request... What happens when we go beyond this allowance? We throw the rejected exception? When would there be more than the configured number of // reads going on?

          Should these be public?

          + public ThreadPoolExecutor getParallelReadsThreadPool()

          { + return parallelReadsThreadPool; + }

          +
          + public DFSQuorumReadMetrics getQuorumReadMetrics() {

          Will have to rename this class? DFSQuorumReadMetrics

          Add class comment on +public class DFSQuorumReadMetrics { saying what the metric means (since there is no description for the metrics it seems).

          Need a space in here? "+public class DFSQuorumReadMetrics {"

          I need to spend more time on the DFSClient changes but above should do for a first cut at a review. Thanks Liang Xie

          Show
          stack stack added a comment - This looks like copy paste issue: + // assert that there were no quorum reads. 60ms + delta < 100ms + assertTrue(metrics.getParallelReadOps() > 0); Here you are asserting that there IS // 'hedged' reads going on. I was wondering if 'hedged requests' a good name for this feature and going by the definition from your citation, it is good by me: Hedged requests. A simple way to curb latency variability is to issue the same request to multiple replicas and use the results from whichever replica responds first. We term such requests "hedged requests" because a client first sends one request to the replica believed to be the most appropriate, but then falls back on sending a secondary request after some brief delay. The client cancels remaining outstanding requests once the first result is received. Although naive implementations of this technique typically add unacceptable additional load, many variations exist that give most of the latency-reduction effects while increasing load only modestly. One such approach is to defer sending a secondary request until the first request has been outstanding for more than the 95th-percentile expected latency for this class of requests. This approach limits the additional load to approximately 5% while substantially shortening the latency tail. .... http://cacm.acm.org/magazines/2013/2/160173-the-tail-at-scale/fulltext We'd change this to be allowHedgeReads? + public volatile boolean allowParallelReads = false; This would be hedgedReadThresholdMillis + private volatile long quorumReadThresholdMillis; ... and so on. Or do you see parallel reads as different from hedged reads? ('Tied requests' from your citation). These are public so clients like hbase can tinker with them: + + public void setQuorumReadTimeout( long timeoutMillis) { + this .quorumReadThresholdMillis = timeoutMillis; + } + + public long getQuorumReadTimeout() { + return this .quorumReadThresholdMillis; + } + + public void enableParallelReads() { + allowParallelReads = true ; + } + + public void disableParallelReads() { + allowParallelReads = false ; + } So if the requests are > the configured number, we run in the current thread which is better than rejecting the request... What happens when we go beyond this allowance? We throw the rejected exception? When would there be more than the configured number of // reads going on? Should these be public? + public ThreadPoolExecutor getParallelReadsThreadPool() { + return parallelReadsThreadPool; + } + + public DFSQuorumReadMetrics getQuorumReadMetrics() { Will have to rename this class? DFSQuorumReadMetrics Add class comment on +public class DFSQuorumReadMetrics { saying what the metric means (since there is no description for the metrics it seems). Need a space in here? "+public class DFSQuorumReadMetrics {" I need to spend more time on the DFSClient changes but above should do for a first cut at a review. Thanks Liang Xie
          Hide
          xieliang007 Liang Xie added a comment -

          Thanks Stack for your so detailed comments !
          Attached v3 addressed the naming related comments firstly.
          And i have got some perf number and would like to share here:

          Test Env:
          Hadoop 2.0 + HBase 0.94.11
          3 datanodes and each DN has only one disk for dfs read/write(yes, only one SATA disk, it's a little poor, haha, but very perfect for current test scenario, since we do want to see the result while bad pread performance occurs)
          one regionserver instance is up and created ycsb test table, loaded 20m records, each row has 3 * 200 bytes, and finally did a major compaction, the webui showed only 1 storefile with 14493MB.
          I use single process ycsb with 10 threads running to do the random read(get) request, each run 10 minutes, and i do clear the hbase block cache and os cache(drop_caches) manually between each testing. the hedged reads thread pool size keeps 50. Here is the detailed result:

          1) dfs.dfsclient.hedged.read.threshold.millis = 500ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 50ms, in deed, it should be very like the current existing impl since per the following result, almost all of response time are less than 500ms, so just very very a few requests probably go to the secondary DN:
          Throughput(ops/sec), 221.8174849820451
          AverageLatency(us), 45055.13540070315
          50thPercentileLatency(us), 24049
          95thPercentileLatency(us), 165905
          99thPercentileLatency(us), 270578

          2) dfs.dfsclient.hedged.read.threshold.millis = 150ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 50ms
          Throughput(ops/sec), 257.6483818568037
          AverageLatency(us), 38781.92033469773
          50thPercentileLatency(us), 20534
          95thPercentileLatency(us), 148194
          99thPercentileLatency(us), 201110

          3) dfs.dfsclient.hedged.read.threshold.millis = 100ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 50ms
          Throughput(ops/sec), 254.35882053973887
          AverageLatency(us), 39291.54205264606
          50thPercentileLatency(us), 20585
          95thPercentileLatency(us), 150998
          99thPercentileLatency(us), 151446

          4) dfs.dfsclient.hedged.read.threshold.millis = 100ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 20ms
          Throughput(ops/sec), 237.20809410260168
          AverageLatency(us), 42110.37126189875
          50thPercentileLatency(us), 20246
          95thPercentileLatency(us), 121147
          99thPercentileLatency(us), 141207

          In summary, in my heavy io-bound random read test scenario, the 99th percentile latency was cut off from 270ms to 141ms via hedged read feature, but it doesn't helpful to improve the avg latency or throughput obviously, this is expected, the biggest benefit is against the long-tail random read latency issue, which is pretty common in HBase.

          Show
          xieliang007 Liang Xie added a comment - Thanks Stack for your so detailed comments ! Attached v3 addressed the naming related comments firstly. And i have got some perf number and would like to share here: Test Env: Hadoop 2.0 + HBase 0.94.11 3 datanodes and each DN has only one disk for dfs read/write(yes, only one SATA disk, it's a little poor, haha, but very perfect for current test scenario, since we do want to see the result while bad pread performance occurs) one regionserver instance is up and created ycsb test table, loaded 20m records, each row has 3 * 200 bytes, and finally did a major compaction, the webui showed only 1 storefile with 14493MB. I use single process ycsb with 10 threads running to do the random read(get) request, each run 10 minutes, and i do clear the hbase block cache and os cache(drop_caches) manually between each testing. the hedged reads thread pool size keeps 50. Here is the detailed result: 1) dfs.dfsclient.hedged.read.threshold.millis = 500ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 50ms, in deed, it should be very like the current existing impl since per the following result, almost all of response time are less than 500ms, so just very very a few requests probably go to the secondary DN: Throughput(ops/sec), 221.8174849820451 AverageLatency(us), 45055.13540070315 50thPercentileLatency(us), 24049 95thPercentileLatency(us), 165905 99thPercentileLatency(us), 270578 2) dfs.dfsclient.hedged.read.threshold.millis = 150ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 50ms Throughput(ops/sec), 257.6483818568037 AverageLatency(us), 38781.92033469773 50thPercentileLatency(us), 20534 95thPercentileLatency(us), 148194 99thPercentileLatency(us), 201110 3) dfs.dfsclient.hedged.read.threshold.millis = 100ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 50ms Throughput(ops/sec), 254.35882053973887 AverageLatency(us), 39291.54205264606 50thPercentileLatency(us), 20585 95thPercentileLatency(us), 150998 99thPercentileLatency(us), 151446 4) dfs.dfsclient.hedged.read.threshold.millis = 100ms, dfs.dfsclient.hedged.read.sleep.interval.millis = 20ms Throughput(ops/sec), 237.20809410260168 AverageLatency(us), 42110.37126189875 50thPercentileLatency(us), 20246 95thPercentileLatency(us), 121147 99thPercentileLatency(us), 141207 In summary, in my heavy io-bound random read test scenario, the 99th percentile latency was cut off from 270ms to 141ms via hedged read feature, but it doesn't helpful to improve the avg latency or throughput obviously, this is expected, the biggest benefit is against the long-tail random read latency issue, which is pretty common in HBase.
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12623929/HDFS-5776-v3.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5920//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5920//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12623929/HDFS-5776-v3.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5920//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5920//console This message is automatically generated.
          Hide
          stack stack added a comment -

          Nice numbers Liang Xie

          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?

          +    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.
          +      }
          +    }
          

          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 Liang Xie

          Show
          stack stack added a comment - Nice numbers Liang Xie 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? + 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. + } + } 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 Liang Xie
          Hide
          cmccabe Colin P. McCabe added a comment -

          Let's keep TestPread as a test of just pread, and have a separate test to test hedged reads.

          +  private ByteBuffer getFirst(ArrayList<Future<ByteBuffer>> futures,
          

          Could we rename this to getFirstToComplete or something like that? getFirst just sounds like it's getting the first element in the ArrayList.

          I think that when the data being read is local, you will not want hedged reads. Let's check for this case.

          Thanks, Liang.

          Show
          cmccabe Colin P. McCabe added a comment - Let's keep TestPread as a test of just pread, and have a separate test to test hedged reads. + private ByteBuffer getFirst(ArrayList<Future<ByteBuffer>> futures, Could we rename this to getFirstToComplete or something like that? getFirst just sounds like it's getting the first element in the ArrayList. I think that when the data being read is local, you will not want hedged reads. Let's check for this case. Thanks, Liang.
          Hide
          stack stack added a comment -

          I think that when the data being read is local, you will not want hedged reads. Let's check for this case.

          You think Mighty Colin P. McCabe? Having hedged read on could ameliorate 'bad sector' syndrome (and its many variants)? Thanks.

          Show
          stack stack added a comment - I think that when the data being read is local, you will not want hedged reads. Let's check for this case. You think Mighty Colin P. McCabe ? Having hedged read on could ameliorate 'bad sector' syndrome (and its many variants)? Thanks.
          Hide
          cmccabe Colin P. McCabe added a comment -

          Having hedged read on could ameliorate 'bad sector' syndrome (and its many variants)? Thanks.

          I guess we probably don't need to special-case local reads, as long as we continue to prefer to read from local datanodes when possible.

          Show
          cmccabe Colin P. McCabe added a comment - Having hedged read on could ameliorate 'bad sector' syndrome (and its many variants)? Thanks. I guess we probably don't need to special-case local reads, as long as we continue to prefer to read from local datanodes when possible.
          Hide
          xieliang007 Liang Xie added a comment -

          Stack, Colin P. McCabe, hedged reads doesn't need to be aware of weather local read or not.
          pread -> fetchBlockByteRange/fetchBlockByteRangeSpeculative -> actualGetFromOneDataNode -> getBlockReader
          we don't need to handle weather local reader or not, the hedged reads only focus on picking a secondary dn, then wait the winner.

          In my test, the region server(rs) instance was in the same box with one dn(say dn1). Take my first case(dfs.dfsclient.hedged.read.threshold.millis = 500ms) for example, in that case, in deed, nearly all of the preads go to the local dn1, i could observe io util% always is 100%(really io bound, aha, btw, i filed HDFS-5727, hope with that we can alleviate the io somehow, i am planning to dig that jira after this done, if you guys have any comments, just put there) during the testing, and dn2/dn3's io util% keep 0%. and in following cases, i could see the io util% of dn2/dn3 began to increment to around 5%~20%.

          Show
          xieliang007 Liang Xie added a comment - Stack , Colin P. McCabe , hedged reads doesn't need to be aware of weather local read or not. pread -> fetchBlockByteRange/fetchBlockByteRangeSpeculative -> actualGetFromOneDataNode -> getBlockReader we don't need to handle weather local reader or not, the hedged reads only focus on picking a secondary dn, then wait the winner. In my test, the region server(rs) instance was in the same box with one dn(say dn1). Take my first case(dfs.dfsclient.hedged.read.threshold.millis = 500ms) for example, in that case, in deed, nearly all of the preads go to the local dn1, i could observe io util% always is 100%(really io bound, aha, btw, i filed HDFS-5727 , hope with that we can alleviate the io somehow, i am planning to dig that jira after this done, if you guys have any comments, just put there) during the testing, and dn2/dn3's io util% keep 0%. and in following cases, i could see the io util% of dn2/dn3 began to increment to around 5%~20%.
          Hide
          xieliang007 Liang Xie added a comment -

          I guess we probably don't need to special-case local reads, as long as we continue to prefer to read from local datanodes when possible.

          yes, correct (hmm, i read several times then fully understant your meaning, my reading english needs to be improvement)

          Show
          xieliang007 Liang Xie added a comment - I guess we probably don't need to special-case local reads, as long as we continue to prefer to read from local datanodes when possible. yes, correct (hmm, i read several times then fully understant your meaning, my reading english needs to be improvement)
          Hide
          xieliang007 Liang Xie added a comment -

          Let's keep TestPread as a test of just pread, and have a separate test to test hedged reads.

          this's what i did, i only changed dfsPreadTest a bit, no creating new conf object, but repy on the input parameter, that's all, i do have separate cases for hedged reads, see testHedgedPreadDFSBasic and testMaxOutHedgedReadPool

          Could we rename this to getFirstToComplete or something like that? getFirst just sounds like it's getting the first element in the ArrayList.

          OK

          I think that when the data being read is local, you will not want hedged reads. Let's check for this case.

          seems you have a different understanding with it, and the latter comments "I guess we probably don't need to special-case local reads, as long as we continue to prefer to read from local datanodes when possible" is correct, there's is no conflict between local read and hedged read. The hedged read still try to request local read if possible, if no successful until the timeout reach, then request to the picked secondary dn, then wait the winner.

          Show
          xieliang007 Liang Xie added a comment - Let's keep TestPread as a test of just pread, and have a separate test to test hedged reads. this's what i did, i only changed dfsPreadTest a bit, no creating new conf object, but repy on the input parameter, that's all, i do have separate cases for hedged reads, see testHedgedPreadDFSBasic and testMaxOutHedgedReadPool Could we rename this to getFirstToComplete or something like that? getFirst just sounds like it's getting the first element in the ArrayList. OK I think that when the data being read is local, you will not want hedged reads. Let's check for this case. seems you have a different understanding with it, and the latter comments "I guess we probably don't need to special-case local reads, as long as we continue to prefer to read from local datanodes when possible" is correct, there's is no conflict between local read and hedged read. The hedged read still try to request local read if possible, if no successful until the timeout reach, then request to the picked secondary dn, then wait the winner.
          Hide
          xieliang007 Liang Xie added a comment -

          Nit: This seems extraneous in the new Callable: + return null;

          the "return null" is needful, since we have a "Future<Void>" definition, if we remove the "return null", the compiler will complain

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

          we need, it returns a object,see Javadoc:

              /**
               * Creates mock object of given class or interface.
               * <p>
               * See examples in javadoc for {@link Mockito} class
               * 
               * @param classToMock class or interface to mock
               * @return mock object
               */
              public static <T> T mock(Class<T> classToMock) {
                  return mock(classToMock, withSettings().defaultAnswer(RETURNS_DEFAULTS));
              }
          
          Show
          xieliang007 Liang Xie added a comment - Nit: This seems extraneous in the new Callable: + return null; the "return null" is needful, since we have a "Future<Void>" definition, if we remove the "return null", the compiler will complain Nit: No need of the intermediary 'instance' assignment – just assign to 'injector'? we need, it returns a object,see Javadoc: /** * Creates mock object of given class or interface . * <p> * See examples in javadoc for {@link Mockito} class * * @param classToMock class or interface to mock * @ return mock object */ public static <T> T mock( Class <T> classToMock) { return mock(classToMock, withSettings().defaultAnswer(RETURNS_DEFAULTS)); }
          Hide
          xieliang007 Liang Xie added a comment -

          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?

          very good suggestion, i modfied the 100ms to 500ms and sleep interval now is 50ms, it should be enough at least to me

          Show
          xieliang007 Liang Xie added a comment - 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? very good suggestion, i modfied the 100ms to 500ms and sleep interval now is 50ms, it should be enough at least to me
          Hide
          xieliang007 Liang Xie added a comment -

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

          OK, add "executor.shutdown()" now, but i didn't put it into finally block, since no exception within it and i don't like to move the executor definition before the try block

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

          the new metrics also useful for HBase side, in our HBase related change, we could gather these new metrics into RegionServerMetrics or some other place

          s/Throw/Throwing/

          done, thanks.

          Needs a space between ie and errMsg?
          + + " from any node: " + ie + errMsg

          good catch, added a space inside getBestNodeErrorString now

          Show
          xieliang007 Liang Xie added a comment - Shut down this executor in the finally? Don't want it sticking around. OK, add "executor.shutdown()" now, but i didn't put it into finally block, since no exception within it and i don't like to move the executor definition before the try block Your nice new metrics showed in your above test? They made sense (I suppose they must basically work since your test relies on them). the new metrics also useful for HBase side, in our HBase related change, we could gather these new metrics into RegionServerMetrics or some other place s/Throw/Throwing/ done, thanks. Needs a space between ie and errMsg? + + " from any node: " + ie + errMsg good catch, added a space inside getBestNodeErrorString now
          Hide
          xieliang007 Liang Xie added a comment -

          getHedgedReadsThreadPool
          Should be kept internal to DFSClient.

          changed from "public" to "protected", since we need it in DFSInputStream for a sanity check.

          So on a dfsclient instance, we can flip hedged reads on and off?

          yes, will only take effect on that instance.

          Should fetchBlockByteRangeSpeculative be called fetchBlockByteRangeHedge or hedgedFetchBlockByteRange

          ok, changed to "hedgedFetchBlockByteRange"

          Show
          xieliang007 Liang Xie added a comment - getHedgedReadsThreadPool Should be kept internal to DFSClient. changed from "public" to "protected", since we need it in DFSInputStream for a sanity check. So on a dfsclient instance, we can flip hedged reads on and off? yes, will only take effect on that instance. Should fetchBlockByteRangeSpeculative be called fetchBlockByteRangeHedge or hedgedFetchBlockByteRange ok, changed to "hedgedFetchBlockByteRange"
          Hide
          xieliang007 Liang Xie added a comment -

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

          There's no guarantee about that, we cann't predict the detailed numHedgedReads count against futures size, there's no direct corelation.

          You need the below new log?
          + DFSClient.LOG.warn("Could not obtain block " + block + errMsg
          + + ". Throw a BlockMissingException");

          yes, it would be better if we have it once diagnoise sth.

          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).

          done

          Show
          xieliang007 Liang Xie added a comment - For sure this will always trigger though the number of futures == numHedgedReads (should futures == numHedgedReads + 1 to be sure?) There's no guarantee about that, we cann't predict the detailed numHedgedReads count against futures size, there's no direct corelation. You need the below new log? + DFSClient.LOG.warn("Could not obtain block " + block + errMsg + + ". Throw a BlockMissingException"); yes, it would be better if we have it once diagnoise sth. 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). done
          Hide
          xieliang007 Liang Xie added a comment -

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

          In bestNode(), if we can not find any candidate, will throw new IOException("No live nodes contain current block"), then chooseDataNode will catch it and retry within a calculated timeWindow and dfsClient.getMaxBlockAcquireFailures limit.

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

          done

          Show
          xieliang007 Liang Xie added a comment - When we get to the end of the pipeline here; i.e. all datanodes have been tried, what happens? In bestNode(), if we can not find any candidate, will throw new IOException("No live nodes contain current block"), then chooseDataNode will catch it and retry within a calculated timeWindow and dfsClient.getMaxBlockAcquireFailures limit. Say why it is ok to ignore the IOE at this point in the comment. done
          Hide
          xieliang007 Liang Xie added a comment -

          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:

          if we hit InvalidEncryptionKeyException or InvalidToken exception, we don't want to addToDeadNodes immediately before retry, so we could not move addToDeadNodes into a finally block, FYI, you could refer to HDFS-5766 for the reason.

          seems i have answered all the above comments, am i missed anyone? let me upload the new one.

          Show
          xieliang007 Liang Xie added a comment - 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: if we hit InvalidEncryptionKeyException or InvalidToken exception, we don't want to addToDeadNodes immediately before retry, so we could not move addToDeadNodes into a finally block, FYI, you could refer to HDFS-5766 for the reason. seems i have answered all the above comments, am i missed anyone? let me upload the new one.
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12624084/HDFS-5776-v4.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5924//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5924//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12624084/HDFS-5776-v4.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5924//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5924//console This message is automatically generated.
          Hide
          enis Enis Soztutar added a comment -

          Nice work Liang. You have beat us to implement this!
          A couple of higher level comments:

          • The numbers look very promising. http://static.googleusercontent.com/media/research.google.com/en/us/people/jeff/Berkeley-Latency-Mar2012.pdf slides 50+ gives some numbers for increased RPC's caused by this. If will be great if we can get some info about this as well.
          • Regarding naming, FB branch calls this quorum reads (which is misleading), and google calls this backup requests. We preferred to use the name "parallel", and "parallel with delay" in design doc for HBASE-10070 (a similar feature in HBase) and in the code we ended up calling it RPC with fallback. It will be very good to use a consistent naming across hdfs and hbase, but not sure which one is better.
          • In getFirstToComplete(), sleeping is not the best practice. It puts an arbitrary delay in returning back, and configuring the sleep timeout is non-trivial. Can we do smt like ExecutorService.invokeAny() or a wait/notify or a coundownLatch design? See http://stackoverflow.com/questions/117690/wait-until-any-of-futuret-is-done
          • Again in the Jeff Dean's slides, they talk about doing the 3rd requests with larger timeout does not buy a lot. Wondering whether we should limit this to only 2 requests or not. Without real-world usage it will be hard to choose one way or the other.
          Show
          enis Enis Soztutar added a comment - Nice work Liang. You have beat us to implement this! A couple of higher level comments: The numbers look very promising. http://static.googleusercontent.com/media/research.google.com/en/us/people/jeff/Berkeley-Latency-Mar2012.pdf slides 50+ gives some numbers for increased RPC's caused by this. If will be great if we can get some info about this as well. Regarding naming, FB branch calls this quorum reads (which is misleading), and google calls this backup requests. We preferred to use the name "parallel", and "parallel with delay" in design doc for HBASE-10070 (a similar feature in HBase) and in the code we ended up calling it RPC with fallback. It will be very good to use a consistent naming across hdfs and hbase, but not sure which one is better. In getFirstToComplete(), sleeping is not the best practice. It puts an arbitrary delay in returning back, and configuring the sleep timeout is non-trivial. Can we do smt like ExecutorService.invokeAny() or a wait/notify or a coundownLatch design? See http://stackoverflow.com/questions/117690/wait-until-any-of-futuret-is-done Again in the Jeff Dean's slides, they talk about doing the 3rd requests with larger timeout does not buy a lot. Wondering whether we should limit this to only 2 requests or not. Without real-world usage it will be hard to choose one way or the other.
          Hide
          stack stack added a comment -

          (Trying to help out w/ naming...)

          Enis Soztutar You don't like 'hedged'? You saw the above citation where there is a 'definition' of hedged reads and then the technique implemented by this patch here. 'backup requests' seems less formal being 'just' a label on a slide. // with delay is accurate but a mouthful.

          Show
          stack stack added a comment - (Trying to help out w/ naming...) Enis Soztutar You don't like 'hedged'? You saw the above citation where there is a 'definition' of hedged reads and then the technique implemented by this patch here. 'backup requests' seems less formal being 'just' a label on a slide. // with delay is accurate but a mouthful.
          Hide
          xieliang007 Liang Xie added a comment -

          Enis Soztutar, thanks for your nice comments.

          sleeping is not the best practice

          Totally agree ! let me try to remove it

          Show
          xieliang007 Liang Xie added a comment - Enis Soztutar , thanks for your nice comments. sleeping is not the best practice Totally agree ! let me try to remove it
          Hide
          xieliang007 Liang Xie added a comment -

          v5 began to use CountDownLatch now, from the test result, looks better for timeout=100ms:

          [OVERALL], Throughput(ops/sec), 275.0043323513337
          [READ], Operations, 165040
          [READ], AverageLatency(us), 36339.62364881241
          [READ], MinLatency(us), 260
          [READ], MaxLatency(us), 1375837
          [READ], 50thPercentileLatency(us), 20306
          [READ], 95thPercentileLatency(us), 115498
          [READ], 99thPercentileLatency(us), 124271

          Show
          xieliang007 Liang Xie added a comment - v5 began to use CountDownLatch now, from the test result, looks better for timeout=100ms: [OVERALL] , Throughput(ops/sec), 275.0043323513337 [READ] , Operations, 165040 [READ] , AverageLatency(us), 36339.62364881241 [READ] , MinLatency(us), 260 [READ] , MaxLatency(us), 1375837 [READ] , 50thPercentileLatency(us), 20306 [READ] , 95thPercentileLatency(us), 115498 [READ] , 99thPercentileLatency(us), 124271
          Hide
          xieliang007 Liang Xie added a comment -

          About naming, personally i'd like to keep "hedged" unchanged, it's not a big issue if HBASE-10070 use another naming, just my thought
          About increased RPC, sure, it's just a tradeoff, we all know that i can give a metric result against diff timeout setting, probably put into HBASE-7509

          doing the 3rd requests with larger timeout does not buy a lot

          Current patch just picks only one secondary dn, thats means there're no more than two requests under normal situation.

          BTW, is it possible let it go into HADOOP-2.4 if all comments are done ? i saw Andrew Wang's post from mail list that there's a plan to release 2.4 in the end of this month, right ?
          If this patch could go in, then we can work immediately at HBASE-7509 with bumping the according dependency version to 2.4, otherwise we need to kick off once it goes into 2.5+, maybe need several weeks ?

          Show
          xieliang007 Liang Xie added a comment - About naming, personally i'd like to keep "hedged" unchanged, it's not a big issue if HBASE-10070 use another naming, just my thought About increased RPC, sure, it's just a tradeoff, we all know that i can give a metric result against diff timeout setting, probably put into HBASE-7509 doing the 3rd requests with larger timeout does not buy a lot Current patch just picks only one secondary dn, thats means there're no more than two requests under normal situation. BTW, is it possible let it go into HADOOP-2 .4 if all comments are done ? i saw Andrew Wang 's post from mail list that there's a plan to release 2.4 in the end of this month, right ? If this patch could go in, then we can work immediately at HBASE-7509 with bumping the according dependency version to 2.4, otherwise we need to kick off once it goes into 2.5+, maybe need several weeks ?
          Hide
          xieliang007 Liang Xie added a comment -

          Added a bit codes into YCSB to get 99.9thPercentile latency, here is the result:
          dfs.dfsclient.hedged.read.threshold.millis=100ms:

          [READ], 95thPercentileLatency(us), 115973
          [READ], 99thPercentileLatency(us), 124829
          [READ], 99.9thPercentileLatency(us), 217892

          dfs.dfsclient.hedged.read.threshold.millis=600000ms(this's equals to disable hedged read feature, since my test duration is 600s):

          [READ], 95thPercentileLatency(us), 149355
          [READ], 99thPercentileLatency(us), 256987
          [READ], 99.9thPercentileLatency(us), 418950

          In practice, maybe we should set the threshold equals to 95/99th percentile latency, here just a test to make the difference more obviously...

          Show
          xieliang007 Liang Xie added a comment - Added a bit codes into YCSB to get 99.9thPercentile latency, here is the result: dfs.dfsclient.hedged.read.threshold.millis=100ms: [READ] , 95thPercentileLatency(us), 115973 [READ] , 99thPercentileLatency(us), 124829 [READ] , 99.9thPercentileLatency(us), 217892 dfs.dfsclient.hedged.read.threshold.millis=600000ms(this's equals to disable hedged read feature, since my test duration is 600s): [READ] , 95thPercentileLatency(us), 149355 [READ] , 99thPercentileLatency(us), 256987 [READ] , 99.9thPercentileLatency(us), 418950 In practice, maybe we should set the threshold equals to 95/99th percentile latency, here just a test to make the difference more obviously...
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12624294/HDFS-5776-v5.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5934//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5934//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12624294/HDFS-5776-v5.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5934//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5934//console This message is automatically generated.
          Hide
          enis Enis Soztutar added a comment -

          About naming, personally i'd like to keep "hedged" unchanged

          Ok, honestly, I am not a native speaker so I had to look it up. But if this is common usage, fine with me.

          v5 began to use CountDownLatch now, from the test result, looks better for timeout=100ms

          Great, thanks.

          Current patch just picks only one secondary dn, thats means there're no more than two requests under normal situation.

          I thought the while(true) loop will continue sending the rpc to 3rd replica. Let me check the patch again.

          Show
          enis Enis Soztutar added a comment - About naming, personally i'd like to keep "hedged" unchanged Ok, honestly, I am not a native speaker so I had to look it up. But if this is common usage, fine with me. v5 began to use CountDownLatch now, from the test result, looks better for timeout=100ms Great, thanks. Current patch just picks only one secondary dn, thats means there're no more than two requests under normal situation. I thought the while(true) loop will continue sending the rpc to 3rd replica. Let me check the patch again.
          Hide
          cmccabe Colin P. McCabe added a comment -

          I like the idea of calling this "hedged reads." We already have a test called TestParallelReads in HDFS which is not related to this, and so it would be a bit confusing to call this feature "parallel reads."

          Show
          cmccabe Colin P. McCabe added a comment - I like the idea of calling this "hedged reads." We already have a test called TestParallelReads in HDFS which is not related to this, and so it would be a bit confusing to call this feature "parallel reads."
          Hide
          arpitagarwal Arpit Agarwal added a comment -

          Hi Liang, thanks for this contribution to HDFS!

          I am still reviewing DFSInputStream#hedgedFetchBlockByteRange and the tests but here is some initial feedback.

          1. Does it make sense for DFSClient#hedgedReadsThreadPool to be a static field? The concern is too many thread pools created by multiple clients on the same node.
          2. Related to the previous - what do you think of not exposing the DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE setting at all? Maybe we can just expose a boolean setting to enable it. The reason I prefer not to surface such settings is because it invites abuse (the concern is not with trusted apps like HBase). If we do expose this setting we should at least have an internal upper bound.
          3. DFSClient#allowHedgedReads seems unnecessary since you can just use (hedgedReadsThreadPool == null). Also you can remove #enableHedgedReads and #disableHedgedReads.
          4. For DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS - can we add an inbuilt minimum delay to defeat applications that set it too low or even zero?
          5. DFSInputStream#chooseDataNode - can the call to getBestNodeErrorString go inside the "if (failures >=..." clause?
          6. #fetchBlockByteRange - can we rename retVal to something like addressPair?
          7. Do we still need the while loop still there in actualGetFromOneDataNode? There is already a while loop in fetchBlockByteRange enclosing the call to actualGetFromOneDataNode. Now we have a nested loop.
          8. Maybe I misunderstood the code flow but it looks like the way the while loops are nested it defeats the usage of refetchToken and refetchEncryptionKey. It looks like the intention was to limit the refetch to 1 across all retries, now we can refetch multiple times.
          9. Related to the previous, #actualGetFromOneDataNode, line 1026, - sorry I did not understand why the try-catch was added around the call to fetchBlockAt.
          10. #actualGetFromOneDataNode, line 1010 - we are using an exception to signal retry to the caller. It might be better to return a boolean instead.
          11. #actualGetFromOneDataNode, line 1033 - the call to DFSClient.LOG.warn is deleted. Assume that was unintentional?
          12. Nitpick - some lines have whitespace-only changes.
          Show
          arpitagarwal Arpit Agarwal added a comment - Hi Liang, thanks for this contribution to HDFS! I am still reviewing DFSInputStream#hedgedFetchBlockByteRange and the tests but here is some initial feedback. Does it make sense for DFSClient#hedgedReadsThreadPool to be a static field? The concern is too many thread pools created by multiple clients on the same node. Related to the previous - what do you think of not exposing the DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE setting at all? Maybe we can just expose a boolean setting to enable it. The reason I prefer not to surface such settings is because it invites abuse (the concern is not with trusted apps like HBase). If we do expose this setting we should at least have an internal upper bound. DFSClient#allowHedgedReads seems unnecessary since you can just use ( hedgedReadsThreadPool == null ). Also you can remove #enableHedgedReads and #disableHedgedReads . For DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS - can we add an inbuilt minimum delay to defeat applications that set it too low or even zero? DFSInputStream#chooseDataNode - can the call to getBestNodeErrorString go inside the " if (failures >=... " clause? #fetchBlockByteRange - can we rename retVal to something like addressPair ? Do we still need the while loop still there in actualGetFromOneDataNode ? There is already a while loop in fetchBlockByteRange enclosing the call to actualGetFromOneDataNode . Now we have a nested loop. Maybe I misunderstood the code flow but it looks like the way the while loops are nested it defeats the usage of refetchToken and refetchEncryptionKey . It looks like the intention was to limit the refetch to 1 across all retries, now we can refetch multiple times. Related to the previous, #actualGetFromOneDataNode , line 1026, - sorry I did not understand why the try-catch was added around the call to fetchBlockAt . #actualGetFromOneDataNode , line 1010 - we are using an exception to signal retry to the caller. It might be better to return a boolean instead. #actualGetFromOneDataNode , line 1033 - the call to DFSClient.LOG.warn is deleted. Assume that was unintentional? Nitpick - some lines have whitespace-only changes.
          Hide
          stack stack added a comment -

          Arpit Agarwal Great review (from a bystander). One note on 1.. Is static ever a good idea for sharing resources? But your point of being able to share amongst DFSClient instances is for sure something we should pursue (in another JIRA?). We could pass a common executor in a shared context and we could also keep running lists of black-listed nodes rather than have each stream discover for themselves the dead... and so on

          Show
          stack stack added a comment - Arpit Agarwal Great review (from a bystander). One note on 1.. Is static ever a good idea for sharing resources? But your point of being able to share amongst DFSClient instances is for sure something we should pursue (in another JIRA?). We could pass a common executor in a shared context and we could also keep running lists of black-listed nodes rather than have each stream discover for themselves the dead... and so on
          Hide
          xieliang007 Liang Xie added a comment -

          Arpit Agarwal, thanks for your nice review!

          The concern is too many thread pools created by multiple clients on the same node

          take it easy, the default configuration: pool=0, that means no extra new threads be created by default. if a end user/application enable hedged read, they should know about this

          what do you think of not exposing the DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE setting at all

          IMHO, i personally prefer the current style, it's less risky, we had a bound queue and once reach the queue limit, we force to exec it in current thread. about the "internal upper bound", how much? 5000? 500000? or sth else? i think if enabling this feature explicitly, the end user/application should know a little backgroud at least, right? just like lots of hadoop timeout config parameter, i never see any internal upper bound impl at all... but if you strongly insist on it, i can add.

          DFSClient#allowHedgedReads seems unnecessary

          let's keep it there, it's more easier to understand for developer or end user.

          For DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS - can we add an inbuilt minimum delay to defeat applications that set it too low or even zero

          my opinion is same as the above one. since we don't have any knowledge about end-user's storage configuration, just image if they have a fast flash(with HDFS-2832 enabled), say fusionio, probably one real disk read only cost tens of microseconds, how should we decide a good minimum defeat setting? so i don't like to add it, i totally get your kindly concern

          DFSInputStream#chooseDataNode - can the call to getBestNodeErrorString go inside the "if (failures >=..." clause?

          another log statement also use it, see "DFSClient.LOG.info("Could not obtain " + block.getBlock...", so it's impossible here.

          #fetchBlockByteRange - can we rename retVal to something like addressPair?

          good. let me rename it

          Do we still need the while loop still there in actualGetFromOneDataNode?

          yes, but the loop is very very light, only when some exceptions like AccessControlException/InvalidEncryptionKeyException/InvalidBlockTokenException happened, will do extra loop, and all those have a fast quit mechanism, like refetchToken/refetchEncryptionKey or disableLegacyBlockReaderLocal, so this loop will only be executed just a very few times

          There is already a while loop in fetchBlockByteRange enclosing the call to actualGetFromOneDataNode. Now we have a nested loop.

          In the loop inside fetchBlockByteRange, the responsibily is picking another dn if there's IOException thrown from actualGetFromOneDataNode, so not a fearful nested loop at all, take it easy

          Maybe I misunderstood the code flow but it looks like the way the while loops are nested it defeats the usage of refetchToken and refetchEncryptionKey. It looks like the intention was to limit the refetch to 1 across all retries, now we can refetch multiple times.

          yes, you had a misunderstanding here. that's why i catch IOException fbae around fetchBlockAt. If we don't catch here, there will be always new refetch from outside loop and will have a spin loop

          Related to the previous, #actualGetFromOneDataNode, line 1026, - sorry I did not understand why the try-catch was added around the call to fetchBlockAt.

          hope the above answer could make you clear? hope my poor english doesn't make everything worse, haha

          #actualGetFromOneDataNode, line 1033 - the call to DFSClient.LOG.warn is deleted. Assume that was unintentional?

          Gooood catch!

          Nitpick - some lines have whitespace-only changes.

          i found several unnessessiry whitespaces existing, i just removed them to make more clear.

          Really thanks all for review!!!

          Show
          xieliang007 Liang Xie added a comment - Arpit Agarwal , thanks for your nice review! The concern is too many thread pools created by multiple clients on the same node take it easy, the default configuration: pool=0, that means no extra new threads be created by default. if a end user/application enable hedged read, they should know about this what do you think of not exposing the DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE setting at all IMHO, i personally prefer the current style, it's less risky, we had a bound queue and once reach the queue limit, we force to exec it in current thread. about the "internal upper bound", how much? 5000? 500000? or sth else? i think if enabling this feature explicitly, the end user/application should know a little backgroud at least, right? just like lots of hadoop timeout config parameter, i never see any internal upper bound impl at all... but if you strongly insist on it, i can add. DFSClient#allowHedgedReads seems unnecessary let's keep it there, it's more easier to understand for developer or end user. For DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS - can we add an inbuilt minimum delay to defeat applications that set it too low or even zero my opinion is same as the above one. since we don't have any knowledge about end-user's storage configuration, just image if they have a fast flash(with HDFS-2832 enabled), say fusionio, probably one real disk read only cost tens of microseconds, how should we decide a good minimum defeat setting? so i don't like to add it, i totally get your kindly concern DFSInputStream#chooseDataNode - can the call to getBestNodeErrorString go inside the "if (failures >=..." clause? another log statement also use it, see "DFSClient.LOG.info("Could not obtain " + block.getBlock...", so it's impossible here. #fetchBlockByteRange - can we rename retVal to something like addressPair? good. let me rename it Do we still need the while loop still there in actualGetFromOneDataNode? yes, but the loop is very very light, only when some exceptions like AccessControlException/InvalidEncryptionKeyException/InvalidBlockTokenException happened, will do extra loop, and all those have a fast quit mechanism, like refetchToken/refetchEncryptionKey or disableLegacyBlockReaderLocal, so this loop will only be executed just a very few times There is already a while loop in fetchBlockByteRange enclosing the call to actualGetFromOneDataNode. Now we have a nested loop. In the loop inside fetchBlockByteRange, the responsibily is picking another dn if there's IOException thrown from actualGetFromOneDataNode, so not a fearful nested loop at all, take it easy Maybe I misunderstood the code flow but it looks like the way the while loops are nested it defeats the usage of refetchToken and refetchEncryptionKey. It looks like the intention was to limit the refetch to 1 across all retries, now we can refetch multiple times. yes, you had a misunderstanding here. that's why i catch IOException fbae around fetchBlockAt. If we don't catch here, there will be always new refetch from outside loop and will have a spin loop Related to the previous, #actualGetFromOneDataNode, line 1026, - sorry I did not understand why the try-catch was added around the call to fetchBlockAt. hope the above answer could make you clear? hope my poor english doesn't make everything worse, haha #actualGetFromOneDataNode, line 1033 - the call to DFSClient.LOG.warn is deleted. Assume that was unintentional? Gooood catch! Nitpick - some lines have whitespace-only changes. i found several unnessessiry whitespaces existing, i just removed them to make more clear. Really thanks all for review!!!
          Hide
          jingzhao Jing Zhao added a comment -
          1. In DFSClient, I agree with Arpit that we should remove the allowHedgedReads field and the enable/disable methods. In the current code, whether hedged read is enabled is determined by the initial setting of the hedgedReadThreadPool. If we provide these extra enable/disable methods, what if a user of DFSClient sets 0 to the thread pool size and later call the enableHedgedReads? Unless we have a clear use case to support the usage of the enable/disable methods, I guess we do not need to provide these flexibility here.
            An alternative way to do this is to have an "Allow-Hedged-Reads" configuration, and if it is set to true, we load the number of thread pool and the threshold time. We will provide an isHedgedReadsEnabled method but we will not provide enable/disable methods. I guess this may be easier for users to understand.
          2. Can this scenario be possible? In hedgedFetchBlockByteRange, if we hit the timeout for the first DN, we will add the DN to the ignore list, and call chooseDataNode again. If the first DN is the only DN we can read, we will get IOException from bestNode. Then we will run into a loop where we keep trying to get another DN multiple times (some NN rpc call will even be fired). And during this process the first DN can even return the data. In this scenario I guess we may get a worse performance? Thus I guess we should not trigger hedged read if we find that we cannot (easily) find the second DN for read?
          Show
          jingzhao Jing Zhao added a comment - In DFSClient, I agree with Arpit that we should remove the allowHedgedReads field and the enable/disable methods. In the current code, whether hedged read is enabled is determined by the initial setting of the hedgedReadThreadPool. If we provide these extra enable/disable methods, what if a user of DFSClient sets 0 to the thread pool size and later call the enableHedgedReads? Unless we have a clear use case to support the usage of the enable/disable methods, I guess we do not need to provide these flexibility here. An alternative way to do this is to have an "Allow-Hedged-Reads" configuration, and if it is set to true, we load the number of thread pool and the threshold time. We will provide an isHedgedReadsEnabled method but we will not provide enable/disable methods. I guess this may be easier for users to understand. Can this scenario be possible? In hedgedFetchBlockByteRange, if we hit the timeout for the first DN, we will add the DN to the ignore list, and call chooseDataNode again. If the first DN is the only DN we can read, we will get IOException from bestNode. Then we will run into a loop where we keep trying to get another DN multiple times (some NN rpc call will even be fired). And during this process the first DN can even return the data. In this scenario I guess we may get a worse performance? Thus I guess we should not trigger hedged read if we find that we cannot (easily) find the second DN for read?
          Hide
          cmccabe Colin P. McCabe added a comment -

          One note on 1.. Is static ever a good idea for sharing resources? But your point of being able to share amongst DFSClient instances is for sure something we should pursue (in another JIRA?)

          Unfortunately, the FileContext API creates a new DFSClient instance for each operation that it does. (The older FileSystem API doesn't have this problem, since the DistributedFileSystem object hangs on to the DFSClient for a while.) This means that we do need to put this in a static, for now, or else FileContext users will be constantly destroying and creating thread-pools.

          I have another change pending which creates the concept of a "cache context," where different threads can use different contexts if they like. For now, let's use a static variable, maybe with a TODO.

          Related to the previous - what do you think of not exposing the DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE setting at all? Maybe we can just expose a boolean setting to enable it. The reason I prefer not to surface such settings is because it invites abuse (the concern is not with trusted apps like HBase). If we do expose this setting we should at least have an internal upper bound.

          I don't see why we wouldn't expose this setting. It doesn't give the client the ability to do anything bad it couldn't already do. You can already try to open a zillion files at once in order to attack the NameNode / DataNodes. Preventing denial-of-service attacks is not currently something we try to do. And in the future, if we ever do try to prevent denial-of-service attacks, I don't think having hedged reads makes that any more or less difficult than it would otherwise be.

          Show
          cmccabe Colin P. McCabe added a comment - One note on 1.. Is static ever a good idea for sharing resources? But your point of being able to share amongst DFSClient instances is for sure something we should pursue (in another JIRA?) Unfortunately, the FileContext API creates a new DFSClient instance for each operation that it does. (The older FileSystem API doesn't have this problem, since the DistributedFileSystem object hangs on to the DFSClient for a while.) This means that we do need to put this in a static, for now, or else FileContext users will be constantly destroying and creating thread-pools. I have another change pending which creates the concept of a "cache context," where different threads can use different contexts if they like. For now, let's use a static variable, maybe with a TODO. Related to the previous - what do you think of not exposing the DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE setting at all? Maybe we can just expose a boolean setting to enable it. The reason I prefer not to surface such settings is because it invites abuse (the concern is not with trusted apps like HBase). If we do expose this setting we should at least have an internal upper bound. I don't see why we wouldn't expose this setting. It doesn't give the client the ability to do anything bad it couldn't already do. You can already try to open a zillion files at once in order to attack the NameNode / DataNodes . Preventing denial-of-service attacks is not currently something we try to do. And in the future, if we ever do try to prevent denial-of-service attacks, I don't think having hedged reads makes that any more or less difficult than it would otherwise be.
          Hide
          cmccabe Colin P. McCabe added a comment -

          By the way, my previous comment was assuming that the alternative proposed to making the thread-pool static was putting it in DFSClient (not a good option). Another option would be making the thread-pool local to the DFSInputStream. However, this seems like it will tend to create an enormous number of threads, especially for applications like HBase that open many files. So again I would argue it should be static.

          Show
          cmccabe Colin P. McCabe added a comment - By the way, my previous comment was assuming that the alternative proposed to making the thread-pool static was putting it in DFSClient (not a good option). Another option would be making the thread-pool local to the DFSInputStream. However, this seems like it will tend to create an enormous number of threads, especially for applications like HBase that open many files. So again I would argue it should be static.
          Hide
          stack stack added a comment -

          An alternative way to do this is to have an "Allow-Hedged-Reads" configuration, and if it is set to true, we load the number of thread pool and the threshold time. We will provide an isHedgedReadsEnabled method but we will not provide enable/disable methods.

          The reviews are great. On the above, while I can see putting the on/off switch as a DN config., we should allow setting at least the config on when to start the hedge read per DFSCient instance.

          This means that we do need to put this in a static, for now, or else FileContext users will be constantly destroying and creating thread-pools.

          Thanks Colin. Makes sense.

          Show
          stack stack added a comment - An alternative way to do this is to have an "Allow-Hedged-Reads" configuration, and if it is set to true, we load the number of thread pool and the threshold time. We will provide an isHedgedReadsEnabled method but we will not provide enable/disable methods. The reviews are great. On the above, while I can see putting the on/off switch as a DN config., we should allow setting at least the config on when to start the hedge read per DFSCient instance. This means that we do need to put this in a static, for now, or else FileContext users will be constantly destroying and creating thread-pools. Thanks Colin. Makes sense.
          Hide
          xieliang007 Liang Xie added a comment -

          Attached v7 makes the pool static now, please review

          Show
          xieliang007 Liang Xie added a comment - Attached v7 makes the pool static now, please review
          Hide
          stack stack added a comment -

          Liang Xie what you think of the new comments above by the lads?

          Now the executor is static, the number of threads config needs to be NumberOfHBaseOpenFiles X 2 else the feature will not work for all files? Thanks.

          Show
          stack stack added a comment - Liang Xie what you think of the new comments above by the lads? Now the executor is static, the number of threads config needs to be NumberOfHBaseOpenFiles X 2 else the feature will not work for all files? Thanks.
          Hide
          xieliang007 Liang Xie added a comment -

          Can this scenario be possible? In hedgedFetchBlockByteRange, if we hit the timeout for the first DN, we will add the DN to the ignore list, and call chooseDataNode again. If the first DN is the only DN we can read, we will get IOException from bestNode. Then we will run into a loop where we keep trying to get another DN multiple times (some NN rpc call will even be fired). And during this process the first DN can even return the data. In this scenario I guess we may get a worse performance? Thus I guess we should not trigger hedged read if we find that we cannot (easily) find the second DN for read?

          yes,there's possible happen about your case, nice! and a very easy handling method is just introduce a double-check function, say enoughNodesForHedgedRead(LocatedBlock block) into the pread checking code branch

          Show
          xieliang007 Liang Xie added a comment - Can this scenario be possible? In hedgedFetchBlockByteRange, if we hit the timeout for the first DN, we will add the DN to the ignore list, and call chooseDataNode again. If the first DN is the only DN we can read, we will get IOException from bestNode. Then we will run into a loop where we keep trying to get another DN multiple times (some NN rpc call will even be fired). And during this process the first DN can even return the data. In this scenario I guess we may get a worse performance? Thus I guess we should not trigger hedged read if we find that we cannot (easily) find the second DN for read? yes,there's possible happen about your case, nice! and a very easy handling method is just introduce a double-check function, say enoughNodesForHedgedRead(LocatedBlock block) into the pread checking code branch
          Hide
          xieliang007 Liang Xie added a comment -

          v8 add the enoughNodesForHedgedRead() function to sanity check, Stack's comments is great, we definitely need a switch per DFSClient instance.

          the number of threads config needs to be NumberOfHBaseOpenFiles X 2 else the feature will not work for all files

          still works, but probably lots of requests will execute in current thread, that means no latency benefit from hedged read feature. this's is a good requirement that we need a per client instance's switch, such that we can let some instances use this feature, we can control it on demand, right ?

          Show
          xieliang007 Liang Xie added a comment - v8 add the enoughNodesForHedgedRead() function to sanity check, Stack 's comments is great, we definitely need a switch per DFSClient instance. the number of threads config needs to be NumberOfHBaseOpenFiles X 2 else the feature will not work for all files still works, but probably lots of requests will execute in current thread, that means no latency benefit from hedged read feature. this's is a good requirement that we need a per client instance's switch, such that we can let some instances use this feature, we can control it on demand, right ?
          Hide
          stack stack added a comment -

          So, to enable, we set DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE in DN config. Should the number of threads in the hbase case be greater than NUMBER_OF_HBASE_OPEN_FILES (though this is most often an unknown number, one that is changing over the life over the hbase process, and up in the thousands frequently)? Otherwise we could set it some 'sensible' number like 16 and then just watch the metrics this patch also adds. If we are too often running the requests in the current thread because the executor has none to spare then we can up the number of pool threads (though it requires a DN restart, a PITA)? That should work for the first cut at this feature.

          nit: You could declare and assign in the one go rather than postpone the assign to the constructor: HEDGED_READ_METRIC = new DFSHedgedReadMetrics();

          What is your thinking regards the boolean enabling/disabling hedge reads in DFSClient Liang Xie? On the one hand, there is a problem where the setting of pool size is done in DN config yet we have enable/disable hedge reads in the API; if the DN config has a pool size set to 0 then hedged reads are off (as was noted above), and though we may 'enable' hedge reads in the API, we won't be getting the behaviour we think we should be getting. On the other hand, it looks like this boolean could be used 'conserving' resources disabling hedged reads on a per request basis though hedged reads have been marked globally 'on' in the DN? Is that your thinking? I'm inclined to agree with the previous reviewers that this may verge on the 'exotic'. For the first cut at this feature, lets have a global on/off switch with number of threads being the means of constraining how much hedged reading we do?

          Otherwise patch looks great to me.

          Show
          stack stack added a comment - So, to enable, we set DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE in DN config. Should the number of threads in the hbase case be greater than NUMBER_OF_HBASE_OPEN_FILES (though this is most often an unknown number, one that is changing over the life over the hbase process, and up in the thousands frequently)? Otherwise we could set it some 'sensible' number like 16 and then just watch the metrics this patch also adds. If we are too often running the requests in the current thread because the executor has none to spare then we can up the number of pool threads (though it requires a DN restart, a PITA)? That should work for the first cut at this feature. nit: You could declare and assign in the one go rather than postpone the assign to the constructor: HEDGED_READ_METRIC = new DFSHedgedReadMetrics(); What is your thinking regards the boolean enabling/disabling hedge reads in DFSClient Liang Xie ? On the one hand, there is a problem where the setting of pool size is done in DN config yet we have enable/disable hedge reads in the API; if the DN config has a pool size set to 0 then hedged reads are off (as was noted above), and though we may 'enable' hedge reads in the API, we won't be getting the behaviour we think we should be getting. On the other hand, it looks like this boolean could be used 'conserving' resources disabling hedged reads on a per request basis though hedged reads have been marked globally 'on' in the DN? Is that your thinking? I'm inclined to agree with the previous reviewers that this may verge on the 'exotic'. For the first cut at this feature, lets have a global on/off switch with number of threads being the means of constraining how much hedged reading we do? Otherwise patch looks great to me.
          Hide
          arpitagarwal Arpit Agarwal added a comment -

          I don't see why we wouldn't expose this setting. It doesn't give the client the ability to do anything bad it couldn't already do. You can already try to open a zillion files at once in order to attack the NameNode / DataNodes. Preventing denial-of-service attacks is not currently something we try to do. And in the future, if we ever do try to prevent denial-of-service attacks, I don't think having hedged reads makes that any more or less difficult than it would otherwise be.

          Colin P. McCabe I am thinking of carelessly configured settings, not a deliberate dos.

          Show
          arpitagarwal Arpit Agarwal added a comment - I don't see why we wouldn't expose this setting. It doesn't give the client the ability to do anything bad it couldn't already do. You can already try to open a zillion files at once in order to attack the NameNode / DataNodes. Preventing denial-of-service attacks is not currently something we try to do. And in the future, if we ever do try to prevent denial-of-service attacks, I don't think having hedged reads makes that any more or less difficult than it would otherwise be. Colin P. McCabe I am thinking of carelessly configured settings, not a deliberate dos.
          Hide
          arpitagarwal Arpit Agarwal added a comment -

          I reviewed the v8 patch. The implementation of hedgedFetchBlockByteRange looks great. Nice use of synchronization tools to make the code easy to understand.

          how much? 5000? 500000? or sth else? i think if enabling this feature explicitly, the end user/application should know a little backgroud at least, right?
          since we don't have any knowledge about end-user's storage configuration, just image if they have a fast flash(with HDFS-2832 enabled), say fusionio, probably one real disk read only cost tens of microseconds,

          Liang Xie #2 and #4 from my previous comment remain unaddressed.

          Threads are not free. If you really want to provide a user configurable setting for the thread count there should be a limit on the order of 64/128. I leave the exact number to you. The best approach is to use a small multiple of the processor count.

          If an app is not well behaved then the absence of limits can create a positive feedback loop. The slower the storage layer the more threads will get created when the correct behavior under load should be back off. Please add a thread count limit or ideally let’s not expose this setting at all.

          The same goes for the delay. Please add a lower bound. The exact value is up to you. We can always revisit the value if it turns out to be a bottleneck.

          let's keep it there, it's more easier to understand for developer or end user.

          I don't think it helps to have these functions and as Jing pointed out there is no purpose for it. I think it would be best to leave a single config setting i.e. either a boolean or a thread count, and a single method #isHedgedReadsEnabled to query the status of the feature.

          yes, but the loop is very very light, only when some exceptions like AccessControlException/InvalidEncryptionKeyException/InvalidBlockTokenException happened, will do extra loop, and all those have a fast quit mechanism, like refetchToken/refetchEncryptionKey or disableLegacyBlockReaderLocal, so this loop will only be executed just a very few times
          ...
          yes, you had a misunderstanding here. that's why i catch IOException fbae around fetchBlockAt. If we don't catch here, there will be always new refetch from outside loop and will have a spin loop

          I still do not understand how you are guarding against multiple refetch. Previously these counters were initialized outside any loop, now they are being reinitialized inside a loop.

          chooseDataNode(LocatedBlock block) function looks redundant and should be removed.

          Show
          arpitagarwal Arpit Agarwal added a comment - I reviewed the v8 patch. The implementation of hedgedFetchBlockByteRange looks great. Nice use of synchronization tools to make the code easy to understand. how much? 5000? 500000? or sth else? i think if enabling this feature explicitly, the end user/application should know a little backgroud at least, right? since we don't have any knowledge about end-user's storage configuration, just image if they have a fast flash(with HDFS-2832 enabled), say fusionio, probably one real disk read only cost tens of microseconds, Liang Xie #2 and #4 from my previous comment remain unaddressed. Threads are not free. If you really want to provide a user configurable setting for the thread count there should be a limit on the order of 64/128. I leave the exact number to you. The best approach is to use a small multiple of the processor count. If an app is not well behaved then the absence of limits can create a positive feedback loop. The slower the storage layer the more threads will get created when the correct behavior under load should be back off. Please add a thread count limit or ideally let’s not expose this setting at all. The same goes for the delay. Please add a lower bound. The exact value is up to you. We can always revisit the value if it turns out to be a bottleneck. let's keep it there, it's more easier to understand for developer or end user. I don't think it helps to have these functions and as Jing pointed out there is no purpose for it. I think it would be best to leave a single config setting i.e. either a boolean or a thread count, and a single method #isHedgedReadsEnabled to query the status of the feature. yes, but the loop is very very light, only when some exceptions like AccessControlException/InvalidEncryptionKeyException/InvalidBlockTokenException happened, will do extra loop, and all those have a fast quit mechanism, like refetchToken/refetchEncryptionKey or disableLegacyBlockReaderLocal, so this loop will only be executed just a very few times ... yes, you had a misunderstanding here. that's why i catch IOException fbae around fetchBlockAt. If we don't catch here, there will be always new refetch from outside loop and will have a spin loop I still do not understand how you are guarding against multiple refetch. Previously these counters were initialized outside any loop, now they are being reinitialized inside a loop. chooseDataNode(LocatedBlock block) function looks redundant and should be removed.
          Hide
          cmccabe Colin P. McCabe added a comment -

          I might be misunderstanding, but it seems like this should be a client setting, not a datanode setting. Right?

          Show
          cmccabe Colin P. McCabe added a comment - I might be misunderstanding, but it seems like this should be a client setting, not a datanode setting. Right?
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12625003/HDFS-5776-v8.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          -1 eclipse:eclipse. The patch failed to build with eclipse:eclipse.

          -1 findbugs. The patch appears to introduce 1 new Findbugs (version 1.3.9) warnings.

          -1 release audit. The applied patch generated 1 release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//testReport/
          Release audit warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//artifact/trunk/patchprocess/newPatchFindbugsWarningshadoop-hdfs.html
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12625003/HDFS-5776-v8.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. -1 eclipse:eclipse . The patch failed to build with eclipse:eclipse. -1 findbugs . The patch appears to introduce 1 new Findbugs (version 1.3.9) warnings. -1 release audit . The applied patch generated 1 release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//testReport/ Release audit warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt Findbugs warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//artifact/trunk/patchprocess/newPatchFindbugsWarningshadoop-hdfs.html Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5941//console This message is automatically generated.
          Hide
          stack stack added a comment -

          I might be misunderstanding, but it seems like this should be a client setting, not a datanode setting. Right?

          Colin P. McCabe You are correct. I had it wrong. s/restart DN/restart client/regionserver/ in the above. Thanks C.

          Show
          stack stack added a comment - I might be misunderstanding, but it seems like this should be a client setting, not a datanode setting. Right? Colin P. McCabe You are correct. I had it wrong. s/restart DN/restart client/regionserver/ in the above. Thanks C.
          Hide
          xieliang007 Liang Xie added a comment -

          Stack

          If we are too often running the requests in the current thread because the executor has none to spare then we can up the number of pool threads (though it requires a DN restart, a PITA)?

          we don't need to restart DN/RS or sth else, we can modify/introduce a hbase shell script to disable/enable the feature per instance or modify the thread number or other requirements, i think it's feasible, and those works, in deed, are the major task for supporting hedged read in HBase side

          nit: You could declare and assign in the one go rather than postpone the assign to the constructor: HEDGED_READ_METRIC = new DFSHedgedReadMetrics();

          good suggestion, let me fix it in patch v9.

          Arpit Agarwal

          Threads are not free. If you really want to provide a user configurable setting for the thread count there should be a limit on the order of 64/128. I leave the exact number to you.

          Fine, let me introduce a hard code up-limit for DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE to 128.

          The same goes for the delay. Please add a lower bound. The exact value is up to you. We can always revisit the value if it turns out to be a bottleneck.

          Fine, let me introduce a hard code down-limit for DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS to 1ms

          I think it would be best to leave a single config setting i.e. either a boolean or a thread count, and a single method #isHedgedReadsEnabled to query the status of the feature.

          Yes, that would be perfect sometimes, but not works for HBase scenario(the above Stack's consideration is great), since we made the pool "static", and per client view, it's more flexible if we provide instance level disable/enable APIs, so we can archive to use the hbase shell script to control the switch per dfs client instance, that'll be cooler

          I still do not understand how you are guarding against multiple refetch. Previously these counters were initialized outside any loop, now they are being reinitialized inside a loop.

          In actualGetFromOneDatanode(), the refetchToken/refetchEncryptionKey is initialized outside the while (true) loop (see Line 993-996), when we hit InvalidEncryptionKeyException/InvalidBlockTokenException, the refetchToken and refetchEncryptionKey will be decreased by 1, (see refetchEncryptionKey-- and refetchToken-- statement), if the exceptions happened again, the check conditions will be failed definitely(see "e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0" and "refetchToken > 0"), so go to the else clause, that'll execute:

                    String msg = "Failed to connect to " + targetAddr + " for file "
                        + src + " for block " + block.getBlock() + ":" + e;
                    DFSClient.LOG.warn("Connection failure: " + msg, e);
                    addToDeadNodes(chosenNode);
                    throw new IOException(msg);
          

          so later, if we chooseDataNode, that dead node will be ignored. Hopefully this time my description is more clear than before

          chooseDataNode(LocatedBlock block) function looks redundant and should be removed.

          it still be called by blockSeekTo(long) and fetchBlockByteRange(...), yes, we can remove it, let me fix it in patch v9.

          Show
          xieliang007 Liang Xie added a comment - Stack If we are too often running the requests in the current thread because the executor has none to spare then we can up the number of pool threads (though it requires a DN restart, a PITA)? we don't need to restart DN/RS or sth else, we can modify/introduce a hbase shell script to disable/enable the feature per instance or modify the thread number or other requirements, i think it's feasible, and those works, in deed, are the major task for supporting hedged read in HBase side nit: You could declare and assign in the one go rather than postpone the assign to the constructor: HEDGED_READ_METRIC = new DFSHedgedReadMetrics(); good suggestion, let me fix it in patch v9. Arpit Agarwal Threads are not free. If you really want to provide a user configurable setting for the thread count there should be a limit on the order of 64/128. I leave the exact number to you. Fine, let me introduce a hard code up-limit for DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE to 128. The same goes for the delay. Please add a lower bound. The exact value is up to you. We can always revisit the value if it turns out to be a bottleneck. Fine, let me introduce a hard code down-limit for DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS to 1ms I think it would be best to leave a single config setting i.e. either a boolean or a thread count, and a single method #isHedgedReadsEnabled to query the status of the feature. Yes, that would be perfect sometimes, but not works for HBase scenario(the above Stack's consideration is great), since we made the pool "static", and per client view, it's more flexible if we provide instance level disable/enable APIs, so we can archive to use the hbase shell script to control the switch per dfs client instance, that'll be cooler I still do not understand how you are guarding against multiple refetch. Previously these counters were initialized outside any loop, now they are being reinitialized inside a loop. In actualGetFromOneDatanode(), the refetchToken/refetchEncryptionKey is initialized outside the while (true) loop (see Line 993-996), when we hit InvalidEncryptionKeyException/InvalidBlockTokenException, the refetchToken and refetchEncryptionKey will be decreased by 1, (see refetchEncryptionKey-- and refetchToken-- statement), if the exceptions happened again, the check conditions will be failed definitely(see "e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0" and "refetchToken > 0"), so go to the else clause, that'll execute: String msg = "Failed to connect to " + targetAddr + " for file " + src + " for block " + block.getBlock() + ":" + e; DFSClient.LOG.warn( "Connection failure: " + msg, e); addToDeadNodes(chosenNode); throw new IOException(msg); so later, if we chooseDataNode, that dead node will be ignored. Hopefully this time my description is more clear than before chooseDataNode(LocatedBlock block) function looks redundant and should be removed. it still be called by blockSeekTo(long) and fetchBlockByteRange(...), yes, we can remove it, let me fix it in patch v9.
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12625329/HDFS-5776-v9.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5948//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5948//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12625329/HDFS-5776-v9.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5948//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5948//console This message is automatically generated.
          Hide
          cmccabe Colin P. McCabe added a comment -

          Fine, let me introduce a hard code up-limit for DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE to 128.

          Please don't. There's no reason to put arbitrary limits into the code. We don't do this with any other configuration settings. At some point, you have to trust the configuration.

          we don't need to restart DN/RS or sth else, we can modify/introduce a hbase shell script to disable/enable the feature per instance or modify the thread number or other requirements, i think it's feasible, and those works, in deed, are the major task for supporting hedged read in HBase side

          Are you suggesting that we make the thread number setting changeable at runtime? That seems like a good idea, but probably something we should do as a follow-on JIRA.

          Show
          cmccabe Colin P. McCabe added a comment - Fine, let me introduce a hard code up-limit for DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE to 128. Please don't. There's no reason to put arbitrary limits into the code. We don't do this with any other configuration settings. At some point, you have to trust the configuration. we don't need to restart DN/RS or sth else, we can modify/introduce a hbase shell script to disable/enable the feature per instance or modify the thread number or other requirements, i think it's feasible, and those works, in deed, are the major task for supporting hedged read in HBase side Are you suggesting that we make the thread number setting changeable at runtime? That seems like a good idea, but probably something we should do as a follow-on JIRA.
          Hide
          xieliang007 Liang Xie added a comment -

          patch v10 removed the hard code limit per Colin's comments.
          patch v9 has the hard code limit.
          Any more comments or +1? Personally i'd like to let the first cut go to trunk and branch-2 asap, so i can kick off the HBase side change. More detailed disagreement could be resolved in other future JIRAs, right? and since the default pool size is 0, so no obvious foreseeable function/performance hurt against the current existing downstream application.

          Show
          xieliang007 Liang Xie added a comment - patch v10 removed the hard code limit per Colin's comments. patch v9 has the hard code limit. Any more comments or +1? Personally i'd like to let the first cut go to trunk and branch-2 asap, so i can kick off the HBase side change. More detailed disagreement could be resolved in other future JIRAs, right? and since the default pool size is 0, so no obvious foreseeable function/performance hurt against the current existing downstream application.
          Hide
          jingzhao Jing Zhao added a comment -

          Thanks for the work Liang Xie! I will review your latest patch and give my comments tonight (PST).

          Show
          jingzhao Jing Zhao added a comment - Thanks for the work Liang Xie ! I will review your latest patch and give my comments tonight (PST).
          Hide
          arpitagarwal Arpit Agarwal added a comment -

          Yes, that would be perfect sometimes, but not works for HBase scenario(the above Stack's consideration is great), since we made the pool "static", and per client view, it's more flexible if we provide instance level disable/enable APIs, so we can archive to use the hbase shell script to control the switch per dfs client instance, that'll be cooler

          Okay.

          In actualGetFromOneDatanode(), the refetchToken/refetchEncryptionKey is initialized outside the while (true) loop (see Line 993-996), when we hit InvalidEncryptionKeyException/InvalidBlockTokenException, the refetchToken and refetchEncryptionKey will be decreased by 1, (see refetchEncryptionKey-- and refetchToken-- statement), if the exceptions happened again, the check conditions will be failed definitely(see "e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0" and "refetchToken > 0"), so go to the else clause, that'll execute:

          Isn't the call to actualGetFromOneDataNode wrapped in a loop itself? I am talking about the while loop in fetchBlockByteRange. Will that not change the behavior? Maybe it is harmless, I am not sure. I just want us to be clear either way.

          Thanks for adding the thread count limit. If we need more than 128 threads per client process just for backup reads we (hdfs) need to think about proper async rpc. Suggesting a lack of limits ignores the point that it can double the DN load on an already loaded cluster. Also 1ms lower bound for the delay is as good as zero but as long as we have a thread count limit I am okay.

          Minor points that don't need to hold up the checkin:

          1. The test looks like a stress test, i.e. we are hoping that some of the hedged requests will complete before the primary requests. We can create a separate Jira to write a deterministic unit test and it’s fine if someone else picks that up later.
          2. A couple of points from my initial feedback (#10, #12) were missed but again not worth holding the checkin.

          Other than clarifying the loop behavior the v9 patch looks fine to me.

          Thanks again for working with the feedback Liang, this is a nice capability to have in HDFS.

          Show
          arpitagarwal Arpit Agarwal added a comment - Yes, that would be perfect sometimes, but not works for HBase scenario(the above Stack's consideration is great), since we made the pool "static", and per client view, it's more flexible if we provide instance level disable/enable APIs, so we can archive to use the hbase shell script to control the switch per dfs client instance, that'll be cooler Okay. In actualGetFromOneDatanode(), the refetchToken/refetchEncryptionKey is initialized outside the while (true) loop (see Line 993-996), when we hit InvalidEncryptionKeyException/InvalidBlockTokenException, the refetchToken and refetchEncryptionKey will be decreased by 1, (see refetchEncryptionKey-- and refetchToken-- statement), if the exceptions happened again, the check conditions will be failed definitely(see "e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0" and "refetchToken > 0"), so go to the else clause, that'll execute: Isn't the call to actualGetFromOneDataNode wrapped in a loop itself? I am talking about the while loop in fetchBlockByteRange . Will that not change the behavior? Maybe it is harmless, I am not sure. I just want us to be clear either way. Thanks for adding the thread count limit. If we need more than 128 threads per client process just for backup reads we (hdfs) need to think about proper async rpc. Suggesting a lack of limits ignores the point that it can double the DN load on an already loaded cluster. Also 1ms lower bound for the delay is as good as zero but as long as we have a thread count limit I am okay. Minor points that don't need to hold up the checkin: The test looks like a stress test, i.e. we are hoping that some of the hedged requests will complete before the primary requests. We can create a separate Jira to write a deterministic unit test and it’s fine if someone else picks that up later. A couple of points from my initial feedback (#10, #12) were missed but again not worth holding the checkin. Other than clarifying the loop behavior the v9 patch looks fine to me. Thanks again for working with the feedback Liang, this is a nice capability to have in HDFS.
          Hide
          stack stack added a comment -

          Arpit Agarwal Would v10 be palatable? You say OK to v9 above but Colin review would favor v10?

          Liang Xie Can you take care of the other nits raised by Arpit Agarwal

          Good stuff.

          Show
          stack stack added a comment - Arpit Agarwal Would v10 be palatable? You say OK to v9 above but Colin review would favor v10? Liang Xie Can you take care of the other nits raised by Arpit Agarwal Good stuff.
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12625502/HDFS-5776-v10.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5957//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5957//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12625502/HDFS-5776-v10.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5957//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5957//console This message is automatically generated.
          Hide
          xieliang007 Liang Xie added a comment -

          Isn't the call to actualGetFromOneDataNode wrapped in a loop itself? I am talking about the while loop in fetchBlockByteRange. Will that not change the behavior? Maybe it is harmless, I am not sure. I just want us to be clear either way.

          Yes, it doesn't change the whole behavior and harmless, in deed, it's safer than before.
          In the old impl, the refetchToken/refetchEncryptionKey are shared by all nodes from chooseDataNode once key/token exception happened. that means if the first node consumed this retry quota, then if the second or third node hit the key/token exception, clearDataEncryptionKey/fetchBlockAt opeerations will not be called, it's a little unfair
          In the new impl/patch, we make the second or later node have a similar retry quota as the first node, it's more fair to me.
          Anyway, it doesn't change the normal path, just safer/fair to the security-enabled scenario.

          The test looks like a stress test, i.e. we are hoping that some of the hedged requests will complete before the primary requests. We can create a separate Jira to write a deterministic unit test and it’s fine if someone else picks that up later.

          Ok, I can track it later.

          For patch v9 or v10, both are OK with me(though our internal branch use the style without limit), since my original wish is to reduce the HBase's P99 and P99.9 latency, not any difference on this point. V9 is safer but probably need to modify HDFS source code again if hit the hardcode limit(It's difficult to a normal end user). IMHO, the actual/final committer who will commit this JIRA can pick one up. It'll be a pity if lots of guys continue to argue this style and hold on the progress, that doesn't help the downstream HBase project at all.

          Show
          xieliang007 Liang Xie added a comment - Isn't the call to actualGetFromOneDataNode wrapped in a loop itself? I am talking about the while loop in fetchBlockByteRange. Will that not change the behavior? Maybe it is harmless, I am not sure. I just want us to be clear either way. Yes, it doesn't change the whole behavior and harmless, in deed, it's safer than before. In the old impl, the refetchToken/refetchEncryptionKey are shared by all nodes from chooseDataNode once key/token exception happened. that means if the first node consumed this retry quota, then if the second or third node hit the key/token exception, clearDataEncryptionKey/fetchBlockAt opeerations will not be called, it's a little unfair In the new impl/patch, we make the second or later node have a similar retry quota as the first node, it's more fair to me. Anyway, it doesn't change the normal path, just safer/fair to the security-enabled scenario. The test looks like a stress test, i.e. we are hoping that some of the hedged requests will complete before the primary requests. We can create a separate Jira to write a deterministic unit test and it’s fine if someone else picks that up later. Ok, I can track it later. For patch v9 or v10, both are OK with me(though our internal branch use the style without limit), since my original wish is to reduce the HBase's P99 and P99.9 latency, not any difference on this point. V9 is safer but probably need to modify HDFS source code again if hit the hardcode limit(It's difficult to a normal end user). IMHO, the actual/final committer who will commit this JIRA can pick one up. It'll be a pity if lots of guys continue to argue this style and hold on the progress, that doesn't help the downstream HBase project at all.
          Hide
          jingzhao Jing Zhao added a comment -

          it's more flexible if we provide instance level disable/enable APIs, so we can archive to use the hbase shell script to control the switch per dfs client instance, that'll be cooler

          I still have some concern about the current implementation:
          1) we do not check threadpool in enableHedgedReads. This makes it possible that isHedgedReadsEnabled() returns true while hedged read is actually not enabled.
          2) DFSClient#setThreadsNumForHedgedReads allows users to keep changing the size of the thread pool.
          To provide instance level disable/enable APIs, I think maybe we can do the following:
          1) Read the thread pool size configuration only when initializing the thread pool, and the size should be >0 and cannot be changed.
          2) Add an "Allow-Hedged-Reads" configuration. Each DFSClient instance reads this configuration, and if it is true, checks and initializes the thread pool if necessary. Users can turn on/off the switch using the enable/disable methods. In the enable method, we check and initialize the thread pool if necessary.

          What do you think Liang Xie?

          Show
          jingzhao Jing Zhao added a comment - it's more flexible if we provide instance level disable/enable APIs, so we can archive to use the hbase shell script to control the switch per dfs client instance, that'll be cooler I still have some concern about the current implementation: 1) we do not check threadpool in enableHedgedReads. This makes it possible that isHedgedReadsEnabled() returns true while hedged read is actually not enabled. 2) DFSClient#setThreadsNumForHedgedReads allows users to keep changing the size of the thread pool. To provide instance level disable/enable APIs, I think maybe we can do the following: 1) Read the thread pool size configuration only when initializing the thread pool, and the size should be >0 and cannot be changed. 2) Add an "Allow-Hedged-Reads" configuration. Each DFSClient instance reads this configuration, and if it is true, checks and initializes the thread pool if necessary. Users can turn on/off the switch using the enable/disable methods. In the enable method, we check and initialize the thread pool if necessary. What do you think Liang Xie ?
          Hide
          jingzhao Jing Zhao added a comment -

          Another thing for enoughNodesForHedgedRead. The current patch checks enoughNodesForHedgedRead before calling hedgedFetchBlockByteRange. Since the deadnodes keeps being updated while reading, we may still hit the issue where we could not easily find the second DN for reading. I think a better way is to add this check in chooseDataNode: if chooseDataNode finds that this is for seeking the second DN (if ignored is not null), and it could not immediately/easily find a DN, the chooseDataNode should skip retrying and we may want to fall back to the normal read.

          Show
          jingzhao Jing Zhao added a comment - Another thing for enoughNodesForHedgedRead. The current patch checks enoughNodesForHedgedRead before calling hedgedFetchBlockByteRange. Since the deadnodes keeps being updated while reading, we may still hit the issue where we could not easily find the second DN for reading. I think a better way is to add this check in chooseDataNode: if chooseDataNode finds that this is for seeking the second DN (if ignored is not null), and it could not immediately/easily find a DN, the chooseDataNode should skip retrying and we may want to fall back to the normal read.
          Hide
          xieliang007 Liang Xie added a comment -

          Could we create another JIRA to track those disagreement? I have said more than three times: the default pool size is 0, so no hurt for all of existing applications by default. I guess it's possible cost one week, one month even one year to argue them...
          Thanks

          Show
          xieliang007 Liang Xie added a comment - Could we create another JIRA to track those disagreement? I have said more than three times: the default pool size is 0, so no hurt for all of existing applications by default. I guess it's possible cost one week, one month even one year to argue them... Thanks
          Hide
          arpitagarwal Arpit Agarwal added a comment -

          stack I am basically +1 on the v9 patch at this point but v10 is a step back. We need a throttle on unbounded thread growth and threadpool size is the most trivial to add. We can file a separate Jira to replace the thread pool limit with something more sophisticated e.g. the client can keep a dynamic estimate of the 95th percentile latency and use that instead of a fixed value from configuration.

          Jing mentioned some issues that look fairly easy to address.

          In the old impl, the refetchToken/refetchEncryptionKey are shared by all nodes from chooseDataNode once key/token exception happened. that means if the first node consumed this retry quota, then if the second or third node hit the key/token exception, clearDataEncryptionKey/fetchBlockAt opeerations will not be called, it's a little unfair

          Liang Xie That makes sense, thanks for the clarification.

          Show
          arpitagarwal Arpit Agarwal added a comment - stack I am basically +1 on the v9 patch at this point but v10 is a step back. We need a throttle on unbounded thread growth and threadpool size is the most trivial to add. We can file a separate Jira to replace the thread pool limit with something more sophisticated e.g. the client can keep a dynamic estimate of the 95th percentile latency and use that instead of a fixed value from configuration. Jing mentioned some issues that look fairly easy to address. In the old impl, the refetchToken/refetchEncryptionKey are shared by all nodes from chooseDataNode once key/token exception happened. that means if the first node consumed this retry quota, then if the second or third node hit the key/token exception, clearDataEncryptionKey/fetchBlockAt opeerations will not be called, it's a little unfair Liang Xie That makes sense, thanks for the clarification.
          Hide
          sureshms Suresh Srinivas added a comment -

          Could we create another JIRA to track those disagreement? I have said more than three times: the default pool size is 0, so no hurt for all of existing applications by default.

          The fact that the issue is brought up many times means that there is an issue that needs to be discussed and resolved.

          I guess it's possible cost one week, one month even one year to argue them...

          If takes more time, so be it. There are many committers who have spent time reviewing and commenting. I understand this is an important feature and the need to get it done sooner. But the core issues must be solved in this jira instead of pushing it to another jira.

          Show
          sureshms Suresh Srinivas added a comment - Could we create another JIRA to track those disagreement? I have said more than three times: the default pool size is 0, so no hurt for all of existing applications by default. The fact that the issue is brought up many times means that there is an issue that needs to be discussed and resolved. I guess it's possible cost one week, one month even one year to argue them... If takes more time, so be it. There are many committers who have spent time reviewing and commenting. I understand this is an important feature and the need to get it done sooner. But the core issues must be solved in this jira instead of pushing it to another jira.
          Hide
          cmccabe Colin P. McCabe added a comment -

          Arpit Agarwal : if I understand your comments correctly, you are concerned that hedged reads may spawn too many threads. But that's why dfs.client.hedged.read.threadpool.size exists. The DFSClient will not create more threads than this.

          We do not check other configuration settings to see if they are "reasonable." For example, if someone wants to set dfs.balancer.dispatcherThreads, dfs.balancer.moverThreads, or dfs.datanode.max.transfer.threads to a zillion, we don't complain. If we tried to set hard limits everywhere, people with different needs would have to recompile hadoop to meet those needs.

          Please remember that, if the client wants to, he/she can sit in a loop and call new Thread(...). It's not like by giving users the ability to control the number of threads they use, we are opening up some new world of security vulnerabilities. The ability for the client to create any number of threads already exists. And it only inconveniences one person: the client themselves.

          Suresh Srinivas: I agree that we should figure out the configuration issues here rather than changing the configuration in an incompatible way later. Jing suggested adding "an Allow-Hedged-Reads configuration" boolean. That certainly seems to solve the problem of having different threads use different settings. Is there any objection, besides the inelegance of having two configs rather than one?

          Show
          cmccabe Colin P. McCabe added a comment - Arpit Agarwal : if I understand your comments correctly, you are concerned that hedged reads may spawn too many threads. But that's why dfs.client.hedged.read.threadpool.size exists. The DFSClient will not create more threads than this. We do not check other configuration settings to see if they are "reasonable." For example, if someone wants to set dfs.balancer.dispatcherThreads , dfs.balancer.moverThreads , or dfs.datanode.max.transfer.threads to a zillion, we don't complain. If we tried to set hard limits everywhere, people with different needs would have to recompile hadoop to meet those needs. Please remember that, if the client wants to, he/she can sit in a loop and call new Thread(...) . It's not like by giving users the ability to control the number of threads they use, we are opening up some new world of security vulnerabilities. The ability for the client to create any number of threads already exists. And it only inconveniences one person: the client themselves. Suresh Srinivas : I agree that we should figure out the configuration issues here rather than changing the configuration in an incompatible way later. Jing suggested adding "an Allow-Hedged-Reads configuration" boolean. That certainly seems to solve the problem of having different threads use different settings. Is there any objection, besides the inelegance of having two configs rather than one?
          Hide
          sureshms Suresh Srinivas added a comment -

          We do not check other configuration settings to see if they are "reasonable."

          Colin P. McCabe, I agree with the points you have made. Checking for reasonable value for the new config does not seem necessary.

          Show
          sureshms Suresh Srinivas added a comment - We do not check other configuration settings to see if they are "reasonable." Colin P. McCabe , I agree with the points you have made. Checking for reasonable value for the new config does not seem necessary.
          Hide
          stack stack added a comment -

          Thanks lads. We are almost there.

          Liang Xie It is better if we work through the issues here before the patch goes in especially while you have the attention of quality reviewers. From your POV, I'm sure it a little frustrating trying to drive the patch home between differing opinions (The time difference doesn't help either – smile). Try to salve any annoyance with the thought that, though it may appear otherwise, folks here are trying to work together to help get the best patch in. Good on you Liang.

          Liang Xie I'd agree with the last few Jing Zhao review comments. What you think?

          Arpit Agarwal Do you buy Colin P. McCabe's argument? It is good by me. If you agree, lets shift the focus to v10 and leave the v9 style behind.

          Good stuff

          Show
          stack stack added a comment - Thanks lads. We are almost there. Liang Xie It is better if we work through the issues here before the patch goes in especially while you have the attention of quality reviewers. From your POV, I'm sure it a little frustrating trying to drive the patch home between differing opinions (The time difference doesn't help either – smile). Try to salve any annoyance with the thought that, though it may appear otherwise, folks here are trying to work together to help get the best patch in. Good on you Liang. Liang Xie I'd agree with the last few Jing Zhao review comments. What you think? Arpit Agarwal Do you buy Colin P. McCabe 's argument? It is good by me. If you agree, lets shift the focus to v10 and leave the v9 style behind. Good stuff
          Hide
          arpitagarwal Arpit Agarwal added a comment -

          I've stated my concerns but if there is broad consensus we don't need caps I won't hold up the checkin.

          Show
          arpitagarwal Arpit Agarwal added a comment - I've stated my concerns but if there is broad consensus we don't need caps I won't hold up the checkin.
          Hide
          xieliang007 Liang Xie added a comment -

          we do not check threadpool in enableHedgedReads. This makes it possible that isHedgedReadsEnabled() returns true while hedged read is actually not enabled.

          i can change to sth like those if you gys want:

           return allowHedgedReads && (HEDGED_READ_THREAD_POOL != null) && HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0;
          

          what do you think ?

          DFSClient#setThreadsNumForHedgedReads allows users to keep changing the size of the thread pool.

          we definitely need the ability to modify the pool size on the fly, especially for HBase ops.

          Read the thread pool size configuration only when initializing the thread pool, and the size should be >0 and cannot be changed

          Here is the same disagreement, if you guys all still insist on making the pool size readonly, i can reupload a new patch. Per my few previous operation experience, it's absolutely inconvenienced to an system ops/admin.

          Show
          xieliang007 Liang Xie added a comment - we do not check threadpool in enableHedgedReads. This makes it possible that isHedgedReadsEnabled() returns true while hedged read is actually not enabled. i can change to sth like those if you gys want: return allowHedgedReads && (HEDGED_READ_THREAD_POOL != null ) && HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0; what do you think ? DFSClient#setThreadsNumForHedgedReads allows users to keep changing the size of the thread pool. we definitely need the ability to modify the pool size on the fly, especially for HBase ops. Read the thread pool size configuration only when initializing the thread pool, and the size should be >0 and cannot be changed Here is the same disagreement, if you guys all still insist on making the pool size readonly, i can reupload a new patch. Per my few previous operation experience, it's absolutely inconvenienced to an system ops/admin.
          Hide
          xieliang007 Liang Xie added a comment -

          I think a better way is to add this check in chooseDataNode: if chooseDataNode finds that this is for seeking the second DN (if ignored is not null), and it could not immediately/easily find a DN, the chooseDataNode should skip retrying and we may want to fall back to the normal read.

          Yeh, sound reasonable. will look into it later once get chance.
          P.S. i am taking a 8+ days long holiday(China Spring Festival) and probably can not reply or make patch timely, sorry.

          Happy Holiday to all guys, thanks for looking at this JIRA !!!

          Show
          xieliang007 Liang Xie added a comment - I think a better way is to add this check in chooseDataNode: if chooseDataNode finds that this is for seeking the second DN (if ignored is not null), and it could not immediately/easily find a DN, the chooseDataNode should skip retrying and we may want to fall back to the normal read. Yeh, sound reasonable. will look into it later once get chance. P.S. i am taking a 8+ days long holiday(China Spring Festival) and probably can not reply or make patch timely, sorry. Happy Holiday to all guys, thanks for looking at this JIRA !!!
          Hide
          stack stack added a comment -

          what do you think ?

          That looks good to me Liang Xie

          ...making the pool size readonly, i can reupload a new patch.

          We can add back the flexibility in a later issue – i.e. being able to adjust pool size on the fly. I suggest posting a patch where the pool size is read from the configuration and is read-only post construction. It would address an above reviewers concern and I believe address all outstanding concerns.

          Base your revision on v10 if you don't mind.

          Show
          stack stack added a comment - what do you think ? That looks good to me Liang Xie ...making the pool size readonly, i can reupload a new patch. We can add back the flexibility in a later issue – i.e. being able to adjust pool size on the fly. I suggest posting a patch where the pool size is read from the configuration and is read-only post construction. It would address an above reviewers concern and I believe address all outstanding concerns. Base your revision on v10 if you don't mind.
          Hide
          xieliang007 Liang Xie added a comment -

          Attached v11:
          1) modify isHedgedReadsEnabled() to consider pool size as well
          2) modify setThreadsNumForHedgedReads to "private" so can not change the thread number from client side dynamically, and remove " synchronized" also.

          Show
          xieliang007 Liang Xie added a comment - Attached v11: 1) modify isHedgedReadsEnabled() to consider pool size as well 2) modify setThreadsNumForHedgedReads to "private" so can not change the thread number from client side dynamically, and remove " synchronized" also.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12625869/HDFS-5776-v11.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          -1 findbugs. The patch appears to introduce 1 new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs:

          org.apache.hadoop.hdfs.qjournal.client.TestQuorumJournalManager

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5977//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5977//artifact/trunk/patchprocess/newPatchFindbugsWarningshadoop-hdfs.html
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5977//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12625869/HDFS-5776-v11.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. -1 findbugs . The patch appears to introduce 1 new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.qjournal.client.TestQuorumJournalManager +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5977//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HDFS-Build/5977//artifact/trunk/patchprocess/newPatchFindbugsWarningshadoop-hdfs.html Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5977//console This message is automatically generated.
          Hide
          stack stack added a comment -

          Address the findbugs warning.

          Jing Zhao Does this patch address your concerns? (Thanks for the review)

          Show
          stack stack added a comment - Address the findbugs warning. Jing Zhao Does this patch address your concerns? (Thanks for the review)
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12625915/HDFS-5776-v12.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs:

          org.apache.hadoop.hdfs.server.namenode.TestAuditLogs

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5979//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5979//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12625915/HDFS-5776-v12.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.server.namenode.TestAuditLogs +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5979//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5979//console This message is automatically generated.
          Hide
          stack stack added a comment -

          Failure seems unrelated. Let me try again to be sure.

          Show
          stack stack added a comment - Failure seems unrelated. Let me try again to be sure.
          Hide
          jingzhao Jing Zhao added a comment -

          Thanks for updating the patch, Liang Xie and stack.

          stack, so the latest patch changes setThreadsNumForHedgedReads to private and aims to make users unable to "change the thread number from client side dynamically". However, users can still create their own configuration object, change the configuration for thread pool size, create an DFSClient instance, and change the thread number? So I think we may want to make it more clean here. Specifically,

          1. the first DFSClient who tries to enable the hedged read will initialize the thread pool (in the DFSClient constructor or in the enable method), so that the enable can be a real enable
          2. changing of the thread pool size (if it is necessary) should still go through a setThreadsNumForHedgedReads method (instead of the constructor of DFSClient), so that a client cannot silently change the size of the thread pool

          Besides, the current patch has not addressed the comment for enoughNodesForHedgedRead/chooseDataNode.

          Show
          jingzhao Jing Zhao added a comment - Thanks for updating the patch, Liang Xie and stack . stack , so the latest patch changes setThreadsNumForHedgedReads to private and aims to make users unable to "change the thread number from client side dynamically". However, users can still create their own configuration object, change the configuration for thread pool size, create an DFSClient instance, and change the thread number? So I think we may want to make it more clean here. Specifically, the first DFSClient who tries to enable the hedged read will initialize the thread pool (in the DFSClient constructor or in the enable method), so that the enable can be a real enable changing of the thread pool size (if it is necessary) should still go through a setThreadsNumForHedgedReads method (instead of the constructor of DFSClient), so that a client cannot silently change the size of the thread pool Besides, the current patch has not addressed the comment for enoughNodesForHedgedRead/chooseDataNode.
          Hide
          stack stack added a comment -

          Jing Zhao Thanks for the new input. Please help me better understand what you mean by making more clean so we can adjust the patch accordingly.

          Hedged reads are set on or off in the client configuration xml and per DFSClient instance can be enabled/disabled as you go. Yes, you could read code and figure that it is possible to do some heavyweight gymnastics creating your own Configuration – expensive – and a new DFSClient – ditto – if you wanted to work around whatever is out in the configuration xml. That seems fine by me especially as there is no real means of shutting down this access route.

          Pardon me but I do not follow what you are asking for in 1. Maybe you are referring to a 'hole' where if the thread count is <= 0 on construction, the enable will have no effect – and you want it to have an 'effect' post construction?

          For 2., you are suggesting that setThreadsNumForHedgedReads not be private but be available API for the DFSClient to toggle as it sees fit?

          I'll let @liang xie address your enoughNodesForHedgedRead comment.

          Thanks for checking back.

          Show
          stack stack added a comment - Jing Zhao Thanks for the new input. Please help me better understand what you mean by making more clean so we can adjust the patch accordingly. Hedged reads are set on or off in the client configuration xml and per DFSClient instance can be enabled/disabled as you go. Yes, you could read code and figure that it is possible to do some heavyweight gymnastics creating your own Configuration – expensive – and a new DFSClient – ditto – if you wanted to work around whatever is out in the configuration xml. That seems fine by me especially as there is no real means of shutting down this access route. Pardon me but I do not follow what you are asking for in 1. Maybe you are referring to a 'hole' where if the thread count is <= 0 on construction, the enable will have no effect – and you want it to have an 'effect' post construction? For 2., you are suggesting that setThreadsNumForHedgedReads not be private but be available API for the DFSClient to toggle as it sees fit? I'll let @liang xie address your enoughNodesForHedgedRead comment. Thanks for checking back.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12625990/HDFS-5776-v12.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs:

          org.apache.hadoop.hdfs.server.namenode.TestNameNodeHttpServer

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5984//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5984//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12625990/HDFS-5776-v12.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.server.namenode.TestNameNodeHttpServer +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5984//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5984//console This message is automatically generated.
          Hide
          jingzhao Jing Zhao added a comment -

          Thanks for the feedback stack.

          My first question is, how will HBase use these enable/disable/setThreadsNumForHedgedReads APIs defined in DFSClient? DFSClient's interface audience is private, and DistributedFileSystem#getClient is also private in HDFS. I have not seen these APIs defined in the DistributedFileSystem/FileContext in the current patch, which means these will be added in a separate jira? In that case, actually we can remove all these API from the current patch and discuss how to define them in that new jira?

          the enable will have no effect

          Yes, if the size of the thread pool is still 0, after the enableHedgedRead is called, the hedged read will not be really enabled right? This makes this API really confusing. Or we can add a javadoc for this method saying "note: this method may not really enable the hedged read, you still need to check the number of the thread pool..."?

          do some heavyweight gymnastics creating your own Configuration – expensive – and a new DFSClient – ditto

          I assume we can have multiple DFSClient instances here since we want to do enable/disable per DFSClient instance? And calling the Configuration#set method to programmatically change the setting of the thread pool size may not be some heavyweight gymnastics. Thus while we aim to disallow users to change the thread number from client side dynamically, users can easily change the thread pool setting in an existing configuration object and use it when creating the next DFSClient instance?

          For 2, actually I do not quite understand the necessity of changing the thread pool size on the fly. I think we should rename setThreadsNumForHedgedReads to initializeThreadPoolForHedgedReads, and remove the "else" section from that method. But if it is really necessary to support this functionality, let's define a clear setThreadsNumForHedgedReads method instead of silently changing the thread pool size in the constructor of DFSClient.

          Show
          jingzhao Jing Zhao added a comment - Thanks for the feedback stack . My first question is, how will HBase use these enable/disable/setThreadsNumForHedgedReads APIs defined in DFSClient? DFSClient's interface audience is private, and DistributedFileSystem#getClient is also private in HDFS. I have not seen these APIs defined in the DistributedFileSystem/FileContext in the current patch, which means these will be added in a separate jira? In that case, actually we can remove all these API from the current patch and discuss how to define them in that new jira? the enable will have no effect Yes, if the size of the thread pool is still 0, after the enableHedgedRead is called, the hedged read will not be really enabled right? This makes this API really confusing. Or we can add a javadoc for this method saying "note: this method may not really enable the hedged read, you still need to check the number of the thread pool..."? do some heavyweight gymnastics creating your own Configuration – expensive – and a new DFSClient – ditto I assume we can have multiple DFSClient instances here since we want to do enable/disable per DFSClient instance? And calling the Configuration#set method to programmatically change the setting of the thread pool size may not be some heavyweight gymnastics. Thus while we aim to disallow users to change the thread number from client side dynamically, users can easily change the thread pool setting in an existing configuration object and use it when creating the next DFSClient instance? For 2, actually I do not quite understand the necessity of changing the thread pool size on the fly. I think we should rename setThreadsNumForHedgedReads to initializeThreadPoolForHedgedReads, and remove the "else" section from that method. But if it is really necessary to support this functionality, let's define a clear setThreadsNumForHedgedReads method instead of silently changing the thread pool size in the constructor of DFSClient.
          Hide
          stack stack added a comment -

          Jing Zhao Thanks for taking the time to look and the great feedback. On the APIs, you make a good point. I can imagine that the notion is that clients such as HBase would selectively enable this feature given there is an associated 'cost'. An approach where we'd enable it on creation only by jiggering the Configuration we pass seems fine for a first cut at least but would imply unhinging threads == 0 as an indicator of enabledness. Liang Xie What you reckon boss? I could cast a patch this way if you are busy. Would it work for your case? Thanks.

          Show
          stack stack added a comment - Jing Zhao Thanks for taking the time to look and the great feedback. On the APIs, you make a good point. I can imagine that the notion is that clients such as HBase would selectively enable this feature given there is an associated 'cost'. An approach where we'd enable it on creation only by jiggering the Configuration we pass seems fine for a first cut at least but would imply unhinging threads == 0 as an indicator of enabledness. Liang Xie What you reckon boss? I could cast a patch this way if you are busy. Would it work for your case? Thanks.
          Hide
          xieliang007 Liang Xie added a comment -

          Stack, yeh, i need your help, thanks! i am really inconvenient to make new patch these two or three days. In the first cut, it should be ok for us HBase.

          Show
          xieliang007 Liang Xie added a comment - Stack , yeh, i need your help, thanks! i am really inconvenient to make new patch these two or three days. In the first cut, it should be ok for us HBase.
          Hide
          stack stack added a comment -

          Addressing Jing Zhao's concerns. Removed all the setters, redo of enoughNodesForHedgedReads, and fix an issue w/ the countdown latch. Not done yet (Test in TestPread fails) and looking to add more....

          Show
          stack stack added a comment - Addressing Jing Zhao's concerns. Removed all the setters, redo of enoughNodesForHedgedReads, and fix an issue w/ the countdown latch. Not done yet (Test in TestPread fails) and looking to add more....
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12627092/HDFS-5776-v13.wip.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          -1 javac. The patch appears to cause the build to fail.

          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6036//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12627092/HDFS-5776-v13.wip.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. -1 javac . The patch appears to cause the build to fail. Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6036//console This message is automatically generated.
          Hide
          xieliang007 Liang Xie added a comment -

          I am from holiday now, will try to catch up the above context firstly then make a new patch shortly

          Show
          xieliang007 Liang Xie added a comment - I am from holiday now, will try to catch up the above context firstly then make a new patch shortly
          Hide
          stack stack added a comment -

          I've been a slacker here Liang Xie. Here is what I have rebased. Not done yet. Needs more work. Let me try and finish it up...

          Show
          stack stack added a comment - I've been a slacker here Liang Xie . Here is what I have rebased. Not done yet. Needs more work. Let me try and finish it up...
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12627780/HDFS-5776-v14.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          -1 javac. The patch appears to cause the build to fail.

          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6087//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12627780/HDFS-5776-v14.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. -1 javac . The patch appears to cause the build to fail. Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6087//console This message is automatically generated.
          Hide
          xieliang007 Liang Xie added a comment -

          Thank you, Michael!

          Show
          xieliang007 Liang Xie added a comment - Thank you, Michael!
          Hide
          xieliang007 Liang Xie added a comment -

          We need to have a NPE check here since the chosenNode be re-init in every while loop:

                  // We got here if exception.  Ignore this node on next go around.
                  if (chosenNode != null) {
                    ignored.add(chosenNode.info);
                  }
          

          After this change, the TestPread case passed locally.
          stack, i found you had resolved Jing Zhao's previous comment about "enoughNodesForHedgedReads", right ? thanks!

          Show
          xieliang007 Liang Xie added a comment - We need to have a NPE check here since the chosenNode be re-init in every while loop: // We got here if exception. Ignore this node on next go around. if (chosenNode != null ) { ignored.add(chosenNode.info); } After this change, the TestPread case passed locally. stack , i found you had resolved Jing Zhao 's previous comment about "enoughNodesForHedgedReads", right ? thanks!
          Hide
          stack stack added a comment -

          Here is as far as I got so far (I added your suggested NPE check and added the missing file). I'm not done yet. Yeah, I'm about addressing Jing Zhao comments Liang Xie Let me finish up....

          Show
          stack stack added a comment - Here is as far as I got so far (I added your suggested NPE check and added the missing file). I'm not done yet. Yeah, I'm about addressing Jing Zhao comments Liang Xie Let me finish up....
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12627920/HDFS-5776-v15.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs:

          org.apache.hadoop.hdfs.server.namenode.ha.TestHASafeMode

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6092//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6092//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12627920/HDFS-5776-v15.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.server.namenode.ha.TestHASafeMode +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6092//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6092//console This message is automatically generated.
          Hide
          stack stack added a comment -

          Address the nice feedback by Jing Zhao. Removed being able to enable/disable/resize post construction of DFSClient and then added handling for case where the pipeline member count could change under us while doing hedged reads because of node death. Tests pass locally. Will try on a cluster now but this posting should be good for review (thanks in advance).

          Show
          stack stack added a comment - Address the nice feedback by Jing Zhao . Removed being able to enable/disable/resize post construction of DFSClient and then added handling for case where the pipeline member count could change under us while doing hedged reads because of node death. Tests pass locally. Will try on a cluster now but this posting should be good for review (thanks in advance).
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12628606/HDFS-5776-v17.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs:

          org.apache.hadoop.hdfs.server.balancer.TestBalancerWithNodeGroup

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6131//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6131//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12628606/HDFS-5776-v17.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.server.balancer.TestBalancerWithNodeGroup +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6131//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6131//console This message is automatically generated.
          Hide
          stack stack added a comment -

          Looks like unrelated failure. Let me resubmit.

          Show
          stack stack added a comment - Looks like unrelated failure. Let me resubmit.
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12628665/HDFS-5776-v17.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6137//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6137//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12628665/HDFS-5776-v17.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6137//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6137//console This message is automatically generated.
          Hide
          jingzhao Jing Zhao added a comment -

          Thanks for updating the patch stack. The latest patch looks great to me. Just some minors:

          1. In DFSClient#initThreadNumForHedgedReads, do we need to check whether HEDGED_READ_THREAD_POOL has already been created?
          2. nit: we may need an @Override annotation here?
            +        new ThreadPoolExecutor.CallerRunsPolicy() {
            +      public void rejectedExecution(Runnable runnable,
            +          ThreadPoolExecutor e) {
            +        LOG.info("Execution rejected, Executing in current thread");
            +        HEDGED_READ_METRIC.incHedgedReadOpsInCurThread();
            +        // will run in the current thread
            +        super.rejectedExecution(runnable, e);
            +      }
            

          +1 after addressing the comments.

          Show
          jingzhao Jing Zhao added a comment - Thanks for updating the patch stack . The latest patch looks great to me. Just some minors: In DFSClient#initThreadNumForHedgedReads, do we need to check whether HEDGED_READ_THREAD_POOL has already been created? nit: we may need an @Override annotation here? + new ThreadPoolExecutor.CallerRunsPolicy() { + public void rejectedExecution( Runnable runnable, + ThreadPoolExecutor e) { + LOG.info( "Execution rejected, Executing in current thread" ); + HEDGED_READ_METRIC.incHedgedReadOpsInCurThread(); + // will run in the current thread + super .rejectedExecution(runnable, e); + } +1 after addressing the comments.
          Hide
          stack stack added a comment -

          Make minimal changes to address @Jing Zhao review comments (the first one is a good catch).

          Show
          stack stack added a comment - Make minimal changes to address @Jing Zhao review comments (the first one is a good catch).
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12629137/HDFS-5776v18.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs:

          org.apache.hadoop.hdfs.server.namenode.TestCacheDirectives

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6160//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6160//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12629137/HDFS-5776v18.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.server.namenode.TestCacheDirectives +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6160//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6160//console This message is automatically generated.
          Hide
          stack stack added a comment -

          This patch has a few small differences that come of some time spent testing:

          1. Adds DEBUG level logging of the one-time setup of the hedged reads pool.
          2. Gives the hedged read pool threads a 'hedged' prefix.
          3. Changes the 'cancel' behavior so it does NOT cancel ongoing reads.

          3. is the biggest change. What I've found is that hdfs reads do not take kindly to being interrupted. The exception types that bubble up are of a few versions – InterruptedIOException, ClosedByInterruptException, and IOEs whose cause is a IE – but I also encountered complaints coming up out of protobuf decoding messages likely because the read was cancelled partway through. Then there was a bunch of logging noise – WARN-level logging – because of the interrupt exceptions and the fact that on interrupt, the node we were reading against would get added to the dead list.

          I had a patch that was more involved dealing w/ the interrupt exceptions and redoing the WARNs but it was getting very involved and I was coming to rely on an untrod path, that of interrupted reads.... so I let it go for now for now.

          This patch lets outstanding reads finish.

          Let me chat w/ Liang Xie to possibly get production numbers on benefit of patch as is.

          Show
          stack stack added a comment - This patch has a few small differences that come of some time spent testing: 1. Adds DEBUG level logging of the one-time setup of the hedged reads pool. 2. Gives the hedged read pool threads a 'hedged' prefix. 3. Changes the 'cancel' behavior so it does NOT cancel ongoing reads. 3. is the biggest change. What I've found is that hdfs reads do not take kindly to being interrupted. The exception types that bubble up are of a few versions – InterruptedIOException, ClosedByInterruptException, and IOEs whose cause is a IE – but I also encountered complaints coming up out of protobuf decoding messages likely because the read was cancelled partway through. Then there was a bunch of logging noise – WARN-level logging – because of the interrupt exceptions and the fact that on interrupt, the node we were reading against would get added to the dead list. I had a patch that was more involved dealing w/ the interrupt exceptions and redoing the WARNs but it was getting very involved and I was coming to rely on an untrod path, that of interrupted reads.... so I let it go for now for now. This patch lets outstanding reads finish. Let me chat w/ Liang Xie to possibly get production numbers on benefit of patch as is.
          Hide
          hadoopqa Hadoop QA added a comment -

          +1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12629871/HDFS-5776v21.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6178//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6178//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12629871/HDFS-5776v21.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/6178//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/6178//console This message is automatically generated.
          Hide
          xieliang007 Liang Xie added a comment -

          Huge thanks, Stack ! Here is the YCSB test result w/ the latest v21 patch:
          My test env, setting, data just same as the previous ones.

          hedged read thread pool size = 0:

          [OVERALL], Throughput(ops/sec), 248.69758063844773
          [READ], AverageLatency(us), 40152.776944654586
          [READ], 50thPercentileLatency(us), 21023
          [READ], 95thPercentileLatency(us), 151130
          [READ], 99thPercentileLatency(us), 255973
          [READ], 99.9thPercentileLatency(us), 420971  
          

          hedged read thread pool size = 50, hedged read timeout = 250ms :

          [OVERALL], Throughput(ops/sec), 231.90953643421446 
          [READ], AverageLatency(us), 43094.41323982726 
          [READ], 50thPercentileLatency(us), 23280
          [READ], 95thPercentileLatency(us), 160024
          [READ], 99thPercentileLatency(us), 258110
          [READ], 99.9thPercentileLatency(us), 278328
          

          hedged read thread pool size = 50, hedged read timeout = 150ms :

          [OVERALL], Throughput(ops/sec), 240.76033437917079
          [READ], AverageLatency(us), 41507.364070480435
          [READ], 50thPercentileLatency(us), 23316
          [READ], 95thPercentileLatency(us), 158907
          [READ], 99thPercentileLatency(us), 168504
          [READ], 99.9thPercentileLatency(us), 218926
          

          hedged read thread pool size = 50, hedged read timeout = 100ms :

          [OVERALL], Throughput(ops/sec), 271.0941444451295
          [READ], AverageLatency(us), 36863.66854547243
          [READ], 50thPercentileLatency(us), 21371
          [READ], 95thPercentileLatency(us), 114943
          [READ], 99thPercentileLatency(us), 121673
          [READ], 99.9thPercentileLatency(us), 195467
          
          Show
          xieliang007 Liang Xie added a comment - Huge thanks, Stack ! Here is the YCSB test result w/ the latest v21 patch: My test env, setting, data just same as the previous ones. hedged read thread pool size = 0: [OVERALL], Throughput(ops/sec), 248.69758063844773 [READ], AverageLatency(us), 40152.776944654586 [READ], 50thPercentileLatency(us), 21023 [READ], 95thPercentileLatency(us), 151130 [READ], 99thPercentileLatency(us), 255973 [READ], 99.9thPercentileLatency(us), 420971 hedged read thread pool size = 50, hedged read timeout = 250ms : [OVERALL], Throughput(ops/sec), 231.90953643421446 [READ], AverageLatency(us), 43094.41323982726 [READ], 50thPercentileLatency(us), 23280 [READ], 95thPercentileLatency(us), 160024 [READ], 99thPercentileLatency(us), 258110 [READ], 99.9thPercentileLatency(us), 278328 hedged read thread pool size = 50, hedged read timeout = 150ms : [OVERALL], Throughput(ops/sec), 240.76033437917079 [READ], AverageLatency(us), 41507.364070480435 [READ], 50thPercentileLatency(us), 23316 [READ], 95thPercentileLatency(us), 158907 [READ], 99thPercentileLatency(us), 168504 [READ], 99.9thPercentileLatency(us), 218926 hedged read thread pool size = 50, hedged read timeout = 100ms : [OVERALL], Throughput(ops/sec), 271.0941444451295 [READ], AverageLatency(us), 36863.66854547243 [READ], 50thPercentileLatency(us), 21371 [READ], 95thPercentileLatency(us), 114943 [READ], 99thPercentileLatency(us), 121673 [READ], 99.9thPercentileLatency(us), 195467
          Hide
          stack stack added a comment -

          Thanks for the numbers Liang Xie I ran some loadings yesterday and little discernible overall difference in spite of my flushing file system cache with regularity (good news, no errors). Today I was going to try and set up measurement of the 99th-percentile, etc... but you did the work. Thanks.

          Hopefully the +1s still stand (If anything, this final patch is more conservative than the one that got the original +1s). I intend to commit this tomorrow unless objection. I will then backport to branch-2.

          Show
          stack stack added a comment - Thanks for the numbers Liang Xie I ran some loadings yesterday and little discernible overall difference in spite of my flushing file system cache with regularity (good news, no errors). Today I was going to try and set up measurement of the 99th-percentile, etc... but you did the work. Thanks. Hopefully the +1s still stand (If anything, this final patch is more conservative than the one that got the original +1s). I intend to commit this tomorrow unless objection. I will then backport to branch-2.
          Hide
          xieliang007 Liang Xie added a comment -

          little discernible overall difference in spite of my flushing file system cache

          you need to have a huge test data size than physical memory, such that lots of HBase read will come to disks, if the disk contend is big enough(e.g. await from iostat reached tens of ms, even hundreds of ms), then the slow disk will make the difference obviously That's why i set up my test env with only one sata disk per dn instance, that will need less test data be loaded to observe a difference.

          Show
          xieliang007 Liang Xie added a comment - little discernible overall difference in spite of my flushing file system cache you need to have a huge test data size than physical memory, such that lots of HBase read will come to disks, if the disk contend is big enough(e.g. await from iostat reached tens of ms, even hundreds of ms), then the slow disk will make the difference obviously That's why i set up my test env with only one sata disk per dn instance, that will need less test data be loaded to observe a difference.
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Hadoop-trunk-Commit #5216 (See https://builds.apache.org/job/Hadoop-trunk-Commit/5216/)
          HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467)

          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
            HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466)
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-trunk-Commit #5216 (See https://builds.apache.org/job/Hadoop-trunk-Commit/5216/ ) HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Hide
          stack stack added a comment -

          What I applied to branch-2 (TestPread needed a little jiggering).

          Show
          stack stack added a comment - What I applied to branch-2 (TestPread needed a little jiggering).
          Hide
          stack stack added a comment -

          Committed to trunk, branch-2 and branch-2.4. Thanks for the sweet feature Liang Xie

          Show
          stack stack added a comment - Committed to trunk, branch-2 and branch-2.4. Thanks for the sweet feature Liang Xie
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Hadoop-Yarn-trunk #492 (See https://builds.apache.org/job/Hadoop-Yarn-trunk/492/)
          HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467)

          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
            HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466)
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-Yarn-trunk #492 (See https://builds.apache.org/job/Hadoop-Yarn-trunk/492/ ) HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Hadoop-Hdfs-trunk #1684 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/1684/)
          HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467)

          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
            HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466)
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-Hdfs-trunk #1684 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/1684/ ) HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Hadoop-Mapreduce-trunk #1709 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/1709/)
          HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467)

          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
            HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466)
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
          • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-Mapreduce-trunk #1709 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/1709/ ) HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571467 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java HDFS-5776 Support 'hedged' reads in DFSClient (stack: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1571466 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
          Hide
          cnauroth Chris Nauroth added a comment -

          FYI, I've discovered that the DFSClient can hang infinitely if using hedged reads and all eligible datanodes die. This bug is present in 2.4.0. I've posted a patch on HDFS-6231 to fix it, hopefully for inclusion in 2.4.1.

          Show
          cnauroth Chris Nauroth added a comment - FYI, I've discovered that the DFSClient can hang infinitely if using hedged reads and all eligible datanodes die. This bug is present in 2.4.0. I've posted a patch on HDFS-6231 to fix it, hopefully for inclusion in 2.4.1.

            People

            • Assignee:
              xieliang007 Liang Xie
              Reporter:
              xieliang007 Liang Xie
            • Votes:
              0 Vote for this issue
              Watchers:
              28 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development