[ 
https://issues.apache.org/jira/browse/HBASE-29254?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Minwoo Kang updated HBASE-29254:
--------------------------------
    Description: 
Let’s assume the data stored in HBase is as follows:

(1) row0/family2:qf1/DeleteColumn
(2) row0/family2:qf1/Put/value2

(3) row1/family1:qf1/Put/value2
(4) row1/family2:qf1/Put/value2

Now, suppose a user starts scanning from {*}row0{*}.

In 
[RegionScannerImpl#nextInternal|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java#L415],
 when the [current 
cell|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java#L446]’s
 row is row0, after reading entry (2) in StoreScanner, if a flush happens, a 
topChanged occurs (Storescanner.peek() is changed where before ...), and the 
value of StoreScanner’s heap.peek() becomes (4) row1/family2:qf1/Put/value2.

Since it is the next row, StoreScanner should return at that point — but it 
fails to recognize that it has moved to the next row because 
[outResult|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java#L835]
 is empty, and ends up including the new row in the result.

Then, in RegionScannerImpl, it sees that nextKv’s row is different from the 
current cell’s row, and returns (since it has moved to a different row).

As a result, even though (3) and (4) belong to the same row (row1), they are 
returned to the client as if they were from different rows.

(3) and (4) should be combined into a single 
[Result|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java],
 but they end up being returned as separate Result instances.

  was:
{color:#000000}For example, suppose the HStore contains the following 
cells:{color}
 
{color:#000000}(1) row1/family:qf1/ts + 2/DeleteColumn/vlen=0/seqid=102/{color}
{color:#000000}(2) row1/family:qf1/ts + 1/Put/vlen=6/seqid=101/value2{color}
{color:#000000}(3) row1/family:qf2/ts + 2/DeleteColumn/vlen=0/seqid=102/{color}
{color:#000000}(4) row1/family:qf2/ts + 1/Put/vlen=6/seqid=101/value2{color}
{color:#000000}(5) row1/family:qf3/ts + 2/DeleteColumn/vlen=0/seqid=102/{color}
{color:#000000}(6) row1/family:qf3/ts + 1/Put/vlen=6/seqid=101/value2{color}
{color:#000000}(7) row2/family:qf1/ts + 3/Put/vlen=6/seqid=103/value1{color}
{color:#000000}(8) row2/family:qf2/ts + 3/Put/vlen=6/seqid=103/value1{color}
{color:#000000}(9) row2/family:qf3/ts + 3/Put/vlen=6/seqid=103/value1{color}
 
{color:#000000}The response from StoreScanner.next(result) should be:{color}
 
{color:#000000}*1. First result:*{color}
{color:#000000}empty{color}
 
*{color:#000000}2. Second result:{color}*
 - row2/family:qf1/ts + 3/Put/vlen=6/seqid=103/value1
 - row2/family:qf2/ts + 3/Put/vlen=6/seqid=103/value1
 - row2/family:qf3/ts + 3/Put/vlen=6/seqid=103/value1

{color:#000000}However, if a flush occurs after reading up to (6), a topChanged 
is triggered, and the response from `StoreScanner.next(result)` becomes:{color} 

 
{color:#000000}*1. First result:*{color}
 - row2/family:qf1/ts + 3/Put/vlen=6/seqid=103/value1
 - row2/family:qf2/ts + 3/Put/vlen=6/seqid=103/value1
 - row2/family:qf3/ts + 3/Put/vlen=6/seqid=103/value1

 
*{color:#000000}2. Second result:{color}*
empty
 
The first result is supposed to return data for `row1`, but it ends up 
including `row2`, which results in the response being split incorrectly across 
user responses.


> StoreScanner returns incorrect row after flush due to topChanged behavior
> -------------------------------------------------------------------------
>
>                 Key: HBASE-29254
>                 URL: https://issues.apache.org/jira/browse/HBASE-29254
>             Project: HBase
>          Issue Type: Bug
>          Components: Scanners
>            Reporter: Minwoo Kang
>            Assignee: Minwoo Kang
>            Priority: Major
>              Labels: pull-request-available
>
> Let’s assume the data stored in HBase is as follows:
> (1) row0/family2:qf1/DeleteColumn
> (2) row0/family2:qf1/Put/value2
> (3) row1/family1:qf1/Put/value2
> (4) row1/family2:qf1/Put/value2
> Now, suppose a user starts scanning from {*}row0{*}.
> In 
> [RegionScannerImpl#nextInternal|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java#L415],
>  when the [current 
> cell|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java#L446]’s
>  row is row0, after reading entry (2) in StoreScanner, if a flush happens, a 
> topChanged occurs (Storescanner.peek() is changed where before ...), and the 
> value of StoreScanner’s heap.peek() becomes (4) row1/family2:qf1/Put/value2.
> Since it is the next row, StoreScanner should return at that point — but it 
> fails to recognize that it has moved to the next row because 
> [outResult|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java#L835]
>  is empty, and ends up including the new row in the result.
> Then, in RegionScannerImpl, it sees that nextKv’s row is different from the 
> current cell’s row, and returns (since it has moved to a different row).
> As a result, even though (3) and (4) belong to the same row (row1), they are 
> returned to the client as if they were from different rows.
> (3) and (4) should be combined into a single 
> [Result|https://github.com/apache/hbase/blob/0b3c17302843d1f4d6f3c6b458f837cb9c274510/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java],
>  but they end up being returned as separate Result instances.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to