You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Jason Tokayer (JIRA)" <ji...@apache.org> on 2017/02/22 17:58:44 UTC

[jira] [Created] (HBASE-17678) ColumnPaginationFilter in a FilterList gives different results when using MUST_PASS_ONE vs MUST_PASS_ALL and a cell has multiple values for a given timestamp

Jason Tokayer created HBASE-17678:
-------------------------------------

             Summary: ColumnPaginationFilter in a FilterList gives different results when using MUST_PASS_ONE vs MUST_PASS_ALL and a cell has multiple values for a given timestamp
                 Key: HBASE-17678
                 URL: https://issues.apache.org/jira/browse/HBASE-17678
             Project: HBase
          Issue Type: Bug
          Components: Filters
    Affects Versions: 1.2.1, 1.3.0
         Environment: RedHat 7.x
            Reporter: Jason Tokayer


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:
{code:none}
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
{code}

Then, use a Scala client as:
{code:none}
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)
{code}

Here are the results for different limit and logicalOp settings:
{code:none}

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
{code}

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.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)