Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-10850

essential column family optimization is broken

VotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Reviewed

    Description

      When using the filter SingleColumnValueFilter, and depending of the columns specified in the scan (filtering column always specified), the results can be different.

      Here is an example.
      Suppose the following table:

      key a:foo a:bar b:foo b:bar
      1 false flag flag flag
      2 true flag flag flag
      3   flag flag flag

      With this filter:

      SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes.toBytes("a"), Bytes.toBytes("foo"), CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("false")));
      filter.setFilterIfMissing(true);
      

      Depending of how I specify the list of columns to add in the scan, the result is different. Yet, all examples below should always return only the first row (key '1'):

      OK:

      scan.addFamily(Bytes.toBytes("a"));
      

      KO (2 results returned, row '3' without 'a:foo' qualifier is returned):

      scan.addFamily(Bytes.toBytes("a"));
      scan.addFamily(Bytes.toBytes("b"));
      

      KO (2 results returned, row '3' without 'a:foo' qualifier is returned):

      scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo"));
      scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("bar"));
      scan.addColumn(Bytes.toBytes("b"), Bytes.toBytes("foo"));
      

      OK:

      scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo"));
      scan.addColumn(Bytes.toBytes("b"), Bytes.toBytes("bar"));
      

      OK:

      scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo"));
      scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("bar"));
      

      This is a regression as it was working properly on HBase 0.92.
      You will find in attachement the unit tests reproducing the issue.

      The analysis of this issue lead us to 2 critical bugs induced in 96 and above versions
      1. The essential family optimization is broken in some cases. In case of condition on some families, we 1st will read those KVs and apply condition on those, when the condition says to filter out that row, we will not go ahead and fetch data from remaining non essential CFs. But now in most of the cases we will do this unwanted data read which is fully against this optimization
      2. We have a CP hook postFilterRow() which will be called when a row is getting filtered out by the Filter. This gives the CP to do a reseek to the next known row which it thinks can evaluate the condition to true. But currently in 96+ code , this hook is not getting called.

      Attachments

        1. 10850-hasFilterRow-v1.txt
          7 kB
          Ted Yu
        2. 10850-hasFilterRow-v2.txt
          8 kB
          Ted Yu
        3. 10850-hasFilterRow-v3.txt
          9 kB
          Ted Yu
        4. 10850-v4.txt
          1 kB
          Ted Yu
        5. 10850-v5.txt
          3 kB
          Ted Yu
        6. 10850-v6.txt
          12 kB
          Ted Yu
        7. 10850-v7.txt
          12 kB
          Ted Yu
        8. HBASE-10850_V2.patch
          4 kB
          Anoop Sam John
        9. HBASE-10850.patch
          1 kB
          haosdent
        10. HBASE-10850-96.patch
          1 kB
          haosdent
        11. HBaseSingleColumnValueFilterTest.java
          8 kB
          Fabien Le Gallo
        12. TestWithMiniCluster.java
          8 kB
          Anoop Sam John

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            yuzhihong@gmail.com Ted Yu
            flegallo Fabien Le Gallo
            Votes:
            0 Vote for this issue
            Watchers:
            12 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment