Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-18410 FilterList Improvement.
  3. HBASE-17678

FilterList with MUST_PASS_ONE may lead to redundant cells returned

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.3.0, 1.2.1, 2.0.0
    • 2.0.0
    • Filters
    • None
    • RedHat 7.x

    Description

      When combining ColumnPaginationFilter with a single-element filterList, MUST_PASS_ONE and MUST_PASS_ALL give different results when there are multiple cells with the same timestamp. This is unexpected since there is only a single filter in the list, and I would believe that MUST_PASS_ALL and MUST_PASS_ONE should only affect the behavior of the joined filter and not the behavior of any one of the individual filters. If this is not a bug then it would be nice if the documentation is updated to explain this nuanced behavior.

      I know that there was a decision made in an earlier Hbase version to keep multiple cells with the same timestamp. This is generally fine but presents an issue when using the aforementioned filter combination.

      Steps to reproduce:
      In the shell create a table and insert some data:

      create 'ns:tbl',{NAME => 'family',VERSIONS => 100}
      put 'ns:tbl','row','family:name','John',1000000000000
      put 'ns:tbl','row','family:name','Jane',1000000000000
      put 'ns:tbl','row','family:name','Gil',1000000000000
      put 'ns:tbl','row','family:name','Jane',1000000000000
      

      Then, use a Scala client as:

      import org.apache.hadoop.hbase.filter._
      import org.apache.hadoop.hbase.util.Bytes
      import org.apache.hadoop.hbase.client._
      import org.apache.hadoop.hbase.{CellUtil, HBaseConfiguration, TableName}
      import scala.collection.mutable._
      
      val config = HBaseConfiguration.create()
      config.set("hbase.zookeeper.quorum", "localhost")
      config.set("hbase.zookeeper.property.clientPort", "2181")
      
      val connection = ConnectionFactory.createConnection(config)
      
      val logicalOp = FilterList.Operator.MUST_PASS_ONE
      val limit = 1
      var resultsList = ListBuffer[String]()
      for (offset <- 0 to 20 by limit) {
      	val table = connection.getTable(TableName.valueOf("ns:tbl"))
      	val paginationFilter = new ColumnPaginationFilter(limit,offset)
      	val filterList: FilterList = new FilterList(logicalOp,paginationFilter)
      	println("@ filterList = "+filterList)
      	val results = table.get(new Get(Bytes.toBytes("row")).setFilter(filterList))
      	val cells = results.rawCells()
      	if (cells != null) {
      		for (cell <- cells) {
      		  val value = new String(CellUtil.cloneValue(cell))
      		  val qualifier = new String(CellUtil.cloneQualifier(cell))
      		  val family = new String(CellUtil.cloneFamily(cell))
      		  val result = "OFFSET = "+offset+":"+family + "," + qualifier + "," + value + "," + cell.getTimestamp()
      		  resultsList.append(result)
      		}
      	}
      }
      resultsList.foreach(println)
      

      Here are the results for different limit and logicalOp settings:

      Limit = 1 & logicalOp = MUST_PASS_ALL:
      scala> resultsList.foreach(println)
      OFFSET = 0:family,name,Jane,1000000000000
      
      Limit = 1 & logicalOp = MUST_PASS_ONE:
      scala> resultsList.foreach(println)
      OFFSET = 0:family,name,Jane,1000000000000
      OFFSET = 1:family,name,Gil,1000000000000
      OFFSET = 2:family,name,Jane,1000000000000
      OFFSET = 3:family,name,John,1000000000000
      
      Limit = 2 & logicalOp = MUST_PASS_ALL:
      scala> resultsList.foreach(println)
      OFFSET = 0:family,name,Jane,1000000000000
      
      Limit = 2 & logicalOp = MUST_PASS_ONE:
      scala> resultsList.foreach(println)
      OFFSET = 0:family,name,Jane,1000000000000
      OFFSET = 2:family,name,Jane,1000000000000
      

      So, it seems that MUST_PASS_ALL gives the expected behavior, but MUST_PASS_ONE does not. Furthermore, MUST_PASS_ONE seems to give only a single (not-duplicated) within a page, but not across pages.

      Attachments

        1. TestColumnPaginationFilterDemo.java
          4 kB
          Zheng Hu
        2. HBASE-17678.v1.rough.patch
          4 kB
          Zheng Hu
        3. HBASE-17678.v1.patch
          7 kB
          Zheng Hu
        4. HBASE-17678.v2.patch
          6 kB
          Zheng Hu
        5. HBASE-17678.v3.patch
          8 kB
          Zheng Hu
        6. HBASE-17678.v4.patch
          12 kB
          Zheng Hu
        7. HBASE-17678.v4.patch
          12 kB
          Zheng Hu
        8. HBASE-17678.v5.patch
          11 kB
          Zheng Hu
        9. HBASE-17678.v6.patch
          12 kB
          Zheng Hu
        10. HBASE-17678.v7.patch
          12 kB
          Zheng Hu
        11. HBASE-17678.v7.patch
          12 kB
          Zheng Hu
        12. HBASE-17678.addendum.patch
          2 kB
          Zheng Hu
        13. HBASE-17678.addendum.patch
          2 kB
          Zheng Hu
        14. HBASE-17678.branch-1.v1.patch
          14 kB
          Zheng Hu
        15. HBASE-17678.branch-1.1.v1.patch
          15 kB
          Zheng Hu
        16. HBASE-17678.branch-1.v1.patch
          14 kB
          Zheng Hu
        17. HBASE-17678.branch-1.v2.patch
          14 kB
          Zheng Hu
        18. HBASE-17678.branch-1.1.v2.patch
          15 kB
          Zheng Hu
        19. HBASE-17678.branch-1.v2.patch
          14 kB
          Zheng Hu
        20. HBASE-17678.branch-1.1.v2.patch
          15 kB
          Zheng Hu
        21. HBASE-17678.branch-1.1.v2.patch
          15 kB
          Zheng Hu

        Issue Links

          Activity

            People

              openinx Zheng Hu
              jason.tokayer Jason Tokayer
              Votes:
              0 Vote for this issue
              Watchers:
              15 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: