HBase
  1. HBase
  2. HBASE-9676

AsyncProcess can create more tasks than hbase.client.max.total.tasks

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Critical Critical
    • Resolution: Fixed
    • Affects Version/s: 0.98.0, 0.96.0
    • Fix Version/s: 0.98.0, 0.96.0
    • Component/s: Client
    • Labels:
      None
    • Hadoop Flags:
      Reviewed

      Description

      hbase.client.max.total.tasks allows to control the number of tasks in progress. But when adding tasks, we take only into account the number of tasks currently running, not the one we're creating.

      1. 9676.v2.patch
        28 kB
        Nicolas Liochon
      2. 9676.v3.patch
        29 kB
        Nicolas Liochon
      3. 9676.v4.patch
        29 kB
        Nicolas Liochon

        Activity

        Hide
        Nicolas Liochon added a comment -

        I've done the following changes:

        • max total task is now defaulted to 100 instead of 100: this seems more suitable to cluster with less than 100 nodes.
        • added some logs
        • added a per server limit (it goes into this patch because controling the number of new tasks depends on it)
        • fixed a flaky test in TestAsync
        Show
        Nicolas Liochon added a comment - I've done the following changes: max total task is now defaulted to 100 instead of 100: this seems more suitable to cluster with less than 100 nodes. added some logs added a per server limit (it goes into this patch because controling the number of new tasks depends on it) fixed a flaky test in TestAsync
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12605888/9676.v2.patch
        against trunk revision .

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

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

        +1 hadoop1.0. The patch compiles against the hadoop 1.0 profile.

        +1 hadoop2.0. The patch compiles against the hadoop 2.0 profile.

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

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

        +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 lineLengths. The patch does not introduce lines longer than 100

        +1 site. The mvn site goal succeeds with this patch.

        -1 core tests. The patch failed these unit tests:
        org.apache.hadoop.hbase.security.access.TestNamespaceCommands

        Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//testReport/
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop1-compat.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
        Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12605888/9676.v2.patch against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 3 new or modified tests. +1 hadoop1.0 . The patch compiles against the hadoop 1.0 profile. +1 hadoop2.0 . The patch compiles against the hadoop 2.0 profile. +1 javadoc . The javadoc tool did not generate any warning messages. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +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 lineLengths . The patch does not introduce lines longer than 100 +1 site . The mvn site goal succeeds with this patch. -1 core tests . The patch failed these unit tests: org.apache.hadoop.hbase.security.access.TestNamespaceCommands Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop1-compat.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/7418//console This message is automatically generated.
        Hide
        Nick Dimiduk added a comment -

        nit vs your comment above: max total tasks is now 100 instead of 200.

        -  private HRegionLocation findDestLocation(Row row, int numAttempt,
        -                                           int posInList, boolean force,
        -                                           Map<String, Boolean> regionStatus) {
        +  private HRegionLocation findDestLocation(Row row, int numAttempt, int posInList) {
        

        Why did you remove the 'force' option?

        -      if (numAttempt > 3 && LOG.isDebugEnabled()) {
        +      if (numAttempt > START_LOG_ERRORS_CNT && LOG.isDebugEnabled()) {
        

        Nit: the new value for START_LOG_ERRORS_CNT is 4, not 3. Was this intentional?

        +      AtomicInteger regionCnt = taskCounterPerRegion.get(encodedRegionName);
        

        I see nbRegion never crossed your mind

        +1

        Show
        Nick Dimiduk added a comment - nit vs your comment above: max total tasks is now 100 instead of 200. - private HRegionLocation findDestLocation(Row row, int numAttempt, - int posInList, boolean force, - Map<String, Boolean> regionStatus) { + private HRegionLocation findDestLocation(Row row, int numAttempt, int posInList) { Why did you remove the 'force' option? - if (numAttempt > 3 && LOG.isDebugEnabled()) { + if (numAttempt > START_LOG_ERRORS_CNT && LOG.isDebugEnabled()) { Nit: the new value for START_LOG_ERRORS_CNT is 4, not 3. Was this intentional? + AtomicInteger regionCnt = taskCounterPerRegion.get(encodedRegionName); I see nbRegion never crossed your mind +1
        Hide
        stack added a comment -

        Is the test failure related?

        I suppose per-server limit makes sense. 5 outstanding requests seems low though. 10?

        Should the change of counts by regions be tied to change of server counts? They are independent at the mo.

        Not sure I understand all going on here. On skim looks good. I like fix to TestAsyncProcess test and constraining by server. +1

        St.Ack

        Show
        stack added a comment - Is the test failure related? I suppose per-server limit makes sense. 5 outstanding requests seems low though. 10? Should the change of counts by regions be tied to change of server counts? They are independent at the mo. Not sure I understand all going on here. On skim looks good. I like fix to TestAsyncProcess test and constraining by server. +1 St.Ack
        Hide
        Nicolas Liochon added a comment -

        Why did you remove the 'force' option?

        Actually, this method was always called with force as true. There are two methods: one w/o force, once w/ force.

        Nit: the new value for START_LOG_ERRORS_CNT is 4, not 3. Was this intentional?

        I may have a bug here: I want to log the retries > 3 and log the sucess > 4. This way if I log an error I will also log the result of the next attempt.

        I see nbRegion never crossed your mind

        For sure

        Is the test failure related?

        I don't thinks so. I will retry locally.

        I suppose per-server limit makes sense. 5 outstanding requests seems low though. 10?

        I will double check the code, but the intent is to have by default:

        • we contact at most 100 servers
        • we don't send more then 1 request per region
        • we don't send more than different 5 requests to the same region servers (but a single request can be about 20 different regions). So with 5 we're already taking 5 handlers out of the 30.

        As well, if we reach the limit per region or per region server, we still send the puts to the other region servers, until we reach the global limit of 100. So we will be able to customize a lot the behavior of the client (for example a map reduce client could be limited to one request per region server).

        I will double check all this and commit tomorrow morning my time if it's trivial. Thanks for the reviews, Nick and Stack.

        Show
        Nicolas Liochon added a comment - Why did you remove the 'force' option? Actually, this method was always called with force as true. There are two methods: one w/o force, once w/ force. Nit: the new value for START_LOG_ERRORS_CNT is 4, not 3. Was this intentional? I may have a bug here: I want to log the retries > 3 and log the sucess > 4. This way if I log an error I will also log the result of the next attempt. I see nbRegion never crossed your mind For sure Is the test failure related? I don't thinks so. I will retry locally. I suppose per-server limit makes sense. 5 outstanding requests seems low though. 10? I will double check the code, but the intent is to have by default: we contact at most 100 servers we don't send more then 1 request per region we don't send more than different 5 requests to the same region servers (but a single request can be about 20 different regions). So with 5 we're already taking 5 handlers out of the 30. As well, if we reach the limit per region or per region server, we still send the puts to the other region servers, until we reach the global limit of 100. So we will be able to customize a lot the behavior of the client (for example a map reduce client could be limited to one request per region server). I will double check all this and commit tomorrow morning my time if it's trivial. Thanks for the reviews, Nick and Stack.
        Hide
        Nicolas Liochon added a comment -

        new version simplifies the new code and adds some comments. I plan to commit it today my time if the tests are ok.

        Show
        Nicolas Liochon added a comment - new version simplifies the new code and adds some comments. I plan to commit it today my time if the tests are ok.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12606065/9676.v3.patch
        against trunk revision .

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

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

        +1 hadoop1.0. The patch compiles against the hadoop 1.0 profile.

        +1 hadoop2.0. The patch compiles against the hadoop 2.0 profile.

        -1 javadoc. The javadoc tool appears to have generated 1 warning messages.

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

        +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 lineLengths. The patch does not introduce lines longer than 100

        +1 site. The mvn site goal succeeds with this patch.

        +1 core tests. The patch passed unit tests in .

        Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//testReport/
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop1-compat.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
        Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12606065/9676.v3.patch against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 3 new or modified tests. +1 hadoop1.0 . The patch compiles against the hadoop 1.0 profile. +1 hadoop2.0 . The patch compiles against the hadoop 2.0 profile. -1 javadoc . The javadoc tool appears to have generated 1 warning messages. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +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 lineLengths . The patch does not introduce lines longer than 100 +1 site . The mvn site goal succeeds with this patch. +1 core tests . The patch passed unit tests in . Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop1-compat.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/7426//console This message is automatically generated.
        Hide
        Nick Dimiduk added a comment -
        +      int newServers = serversIncluded.keySet().size(); // number of servers we're going to contact
        

        To be strictly correct, shouldn't this be the intersection of the set of new servers with the set of serversIncluded.keySet() with counters >= 1?

        Show
        Nick Dimiduk added a comment - + int newServers = serversIncluded.keySet().size(); // number of servers we're going to contact To be strictly correct, shouldn't this be the intersection of the set of new servers with the set of serversIncluded.keySet() with counters >= 1?
        Hide
        Nicolas Liochon added a comment -

        To be strictly correct, shouldn't this be the intersection of the set of new servers with the set of serversIncluded.keySet() with counters >= 1?

        Yeah there is a bug. I'm going to fix this.

        Show
        Nicolas Liochon added a comment - To be strictly correct, shouldn't this be the intersection of the set of new servers with the set of serversIncluded.keySet() with counters >= 1? Yeah there is a bug. I'm going to fix this.
        Hide
        Nicolas Liochon added a comment -

        2 bugs fixed in the last version. Thanks for the catch, Nick.

        Show
        Nicolas Liochon added a comment - 2 bugs fixed in the last version. Thanks for the catch, Nick.
        Hide
        Nick Dimiduk added a comment -
        +  /**
        +   * The number of tasks we run in parallel on a single region.
        +   * With 1 (the default) , we ensure that the ordering of the queries is respected: we don't start
        +   * a set of operations on a region before the previous one is done. As well, this limits
        +   * the pressure we put on the region server.
        +   */
           protected final int maxConcurrentTasksPerRegion;
        

        This documentation is very good. Thinking about consistency models and the linear ordering of writes, I'm concerned about proper documentation of this setting. By setting hbase.client.max.perregion.tasks > 1 the user has accepted the possibility of the RS ACKing writes out of order. Technically this is okay, I think, because at this level, the API only makes guarantees about a single row; so long as edits to a single row are properly grouped, there are no surprises. The user has also accepted the out-of-order nature of writes across multiple regions; conceptually this is an extension.

        I don't see the second bug fixed in this patch; I'll have to trust you.

        +1

        Show
        Nick Dimiduk added a comment - + /** + * The number of tasks we run in parallel on a single region. + * With 1 (the default) , we ensure that the ordering of the queries is respected: we don't start + * a set of operations on a region before the previous one is done. As well, this limits + * the pressure we put on the region server. + */ protected final int maxConcurrentTasksPerRegion; This documentation is very good. Thinking about consistency models and the linear ordering of writes, I'm concerned about proper documentation of this setting. By setting hbase.client.max.perregion.tasks > 1 the user has accepted the possibility of the RS ACKing writes out of order. Technically this is okay, I think, because at this level, the API only makes guarantees about a single row; so long as edits to a single row are properly grouped, there are no surprises. The user has also accepted the out-of-order nature of writes across multiple regions; conceptually this is an extension. I don't see the second bug fixed in this patch; I'll have to trust you. +1
        Hide
        Hadoop QA added a comment -

        +1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12606161/9676.v4.patch
        against trunk revision .

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

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

        +1 hadoop1.0. The patch compiles against the hadoop 1.0 profile.

        +1 hadoop2.0. The patch compiles against the hadoop 2.0 profile.

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

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

        +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 lineLengths. The patch does not introduce lines longer than 100

        +1 site. The mvn site goal succeeds with this patch.

        +1 core tests. The patch passed unit tests in .

        Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//testReport/
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop1-compat.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
        Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
        Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12606161/9676.v4.patch against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 3 new or modified tests. +1 hadoop1.0 . The patch compiles against the hadoop 1.0 profile. +1 hadoop2.0 . The patch compiles against the hadoop 2.0 profile. +1 javadoc . The javadoc tool did not generate any warning messages. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +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 lineLengths . The patch does not introduce lines longer than 100 +1 site . The mvn site goal succeeds with this patch. +1 core tests . The patch passed unit tests in . Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop1-compat.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/7435//console This message is automatically generated.
        Hide
        Nicolas Liochon added a comment -

        Committed to .96 & trunk, thanks for the review, Nick & Stack.

        Show
        Nicolas Liochon added a comment - Committed to .96 & trunk, thanks for the review, Nick & Stack.
        Hide
        Nicolas Liochon added a comment -

        By setting hbase.client.max.perregion.tasks > 1 the user has accepted the possibility of the RS ACKing writes out of order.

        Yes, that's why I set it to 1 by default (and I tend to think that at the end nobody will actually change this). The guarantees are brittle whatever this setting or .94 / .96: if there is a split while you're sending your puts, the last put may be inserted before the first, even if initially they were on the same region.

        I think the total task & the region server config should make it to the hbase-site.xml, but not on the one on the regions...

        Show
        Nicolas Liochon added a comment - By setting hbase.client.max.perregion.tasks > 1 the user has accepted the possibility of the RS ACKing writes out of order. Yes, that's why I set it to 1 by default (and I tend to think that at the end nobody will actually change this). The guarantees are brittle whatever this setting or .94 / .96: if there is a split while you're sending your puts, the last put may be inserted before the first, even if initially they were on the same region. I think the total task & the region server config should make it to the hbase-site.xml, but not on the one on the regions...
        Hide
        Hudson added a comment -

        SUCCESS: Integrated in hbase-0.96 #115 (See https://builds.apache.org/job/hbase-0.96/115/)
        HBASE-9676 AsyncProcess can create more tasks than hbase.client.max.total.tasks (nkeywal: rev 1528341)

        • /hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
        • /hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
        Show
        Hudson added a comment - SUCCESS: Integrated in hbase-0.96 #115 (See https://builds.apache.org/job/hbase-0.96/115/ ) HBASE-9676 AsyncProcess can create more tasks than hbase.client.max.total.tasks (nkeywal: rev 1528341) /hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java /hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
        Hide
        Hudson added a comment -

        FAILURE: Integrated in hbase-0.96-hadoop2 #69 (See https://builds.apache.org/job/hbase-0.96-hadoop2/69/)
        HBASE-9676 AsyncProcess can create more tasks than hbase.client.max.total.tasks (nkeywal: rev 1528341)

        • /hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
        • /hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
        Show
        Hudson added a comment - FAILURE: Integrated in hbase-0.96-hadoop2 #69 (See https://builds.apache.org/job/hbase-0.96-hadoop2/69/ ) HBASE-9676 AsyncProcess can create more tasks than hbase.client.max.total.tasks (nkeywal: rev 1528341) /hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java /hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
        Hide
        Nick Dimiduk added a comment -

        The guarantees are brittle whatever this setting or .94 / .96: if there is a split while you're sending your puts, the last put may be inserted before the first, even if initially they were on the same region.

        Oh, that's true. We need to adopt this tagline: "The only guarantee is that there are no guarantees! (outside of single row operations)." This whole business is best left up to MultiRowMutationEndpoint, then.

        Thanks for clarifying.

        Show
        Nick Dimiduk added a comment - The guarantees are brittle whatever this setting or .94 / .96: if there is a split while you're sending your puts, the last put may be inserted before the first, even if initially they were on the same region. Oh, that's true. We need to adopt this tagline: "The only guarantee is that there are no guarantees! (outside of single row operations)." This whole business is best left up to MultiRowMutationEndpoint, then. Thanks for clarifying.

          People

          • Assignee:
            Nicolas Liochon
            Reporter:
            Nicolas Liochon
          • Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development