[ 
https://issues.apache.org/jira/browse/HBASE-6269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13401383#comment-13401383
 ] 

ShiXing commented on HBASE-6269:
--------------------------------

@anoop

There are 2 StoreFiles after flush two times, the sf2's sequenceId > sf1's 
sequenceId.

When get:
step1. the sf2 is the highest StoreFileScanner, and it enforceSeek() in 
KeyValueHeap.pollRealKV(), so the KeyValue2 is read out from StoreFile by real 
seek. And it compares to the fake KeyValue(called FakeKeyValue) that generated 
by KeyValue.createFirstOnRow() in StoreScanner.next(), and the FakeKeyValue's 
row, family, qualifier, timestamp, memstoreTS(always 0 for StoreFileScanner) 
are the same as KeyValue2 excepts Key type is Maximum, and Key type in 
KeyValue2 is Put. And the {code}comparator.compare(curKV=KeyValue2, 
nextKV=FakeKeyValue) = 251 > 0{code}. It means that the highest 
StoreFileScanner's highest KeyValue is not higher than the second. Followed is 
the value for example
{code}
KeyValue2 : putRow/family:qualifier/1234567/Put/vlen=6/ts=0
FakeKeyValue : putRow/family:qualifier/1234567/Maximum/vlen=0/ts=0
{code}

And then the second highest StoreFileScanner becomes the highest, and the 
highest is added to the heap.

Step2. The sf1's highest KeyValue is read out , we call it KeyValue1, the real 
value is the same as KeyValue2 fetched again by heap.peek():
{code}
KeyValue1 : putRow/family:qualifier/1234567/Put/vlen=6/ts=0
{code}

Step3. KeyValue1 compares KeyValue2, and the 
{code}comparator.compare(curKV=KeyValue1, nextKV=KeyValue2) = 0{code}, and 
return the sf1's scanner as the highest StoreFileScanner.

My solution is that:

If all the highest KeyValue read out from the StoreFileScanners are the 
same(compare return 0), then we should keep the Scanners orig order by 
sequenceId.
                
> Lazyseek should use the maxSequenseId StoreFile's KeyValue as the latest 
> KeyValue
> ---------------------------------------------------------------------------------
>
>                 Key: HBASE-6269
>                 URL: https://issues.apache.org/jira/browse/HBASE-6269
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>    Affects Versions: 0.94.0
>            Reporter: ShiXing
>            Assignee: ShiXing
>         Attachments: HBASE-6269-v1.patch
>
>
> When I fix the bug HBASE-6195, there is happened to find sometimes the test 
> case will fail, https://builds.apache.org/job/HBase-0.94/259/.
> If there are two Put/Increment with same row, family, qualifier, timestamp 
> and different memstoreTS, after each Put/Increment, we do a memstore flush. 
> So there will be two StoreFile with same KeyValue(except memstoreTS and 
> SequenceId).
> When I got the row, I always got the old records, the test case like this:
> {code}
>   public void testPutWithMemStoreFlush() throws Exception {
>     Configuration conf = HBaseConfiguration.create();
>     String method = "testPutWithMemStoreFlush";
>     byte[] tableName = Bytes.toBytes(method);
>     byte[] family = Bytes.toBytes("family");;
>     byte[] qualifier = Bytes.toBytes("qualifier");
>     byte[] row = Bytes.toBytes("putRow");
>     byte[] value = null;
>     this.region = initHRegion(tableName, method, conf, family);
>     Put put = null;
>     Get get = null;
>     List<KeyValue> kvs = null;
>     Result res = null;
>     
>     put = new Put(row);
>     value = Bytes.toBytes("value0");
>     put.add(family, qualifier, 1234567l, value);
>     region.put(put);
>     System.out.print("get value before flush after put value0 : ");
>     get = new Get(row);
>     get.addColumn(family, qualifier);
>     get.setMaxVersions();
>     res = this.region.get(get, null);
>     kvs = res.getColumn(family, qualifier);
>     for (int i = 0; i < kvs.size(); i++) {
>       System.out.println(Bytes.toString(kvs.get(i).getValue()));
>     }
>     region.flushcache();
>     
>     System.out.print("get value after flush after put value0 : ");
>     get = new Get(row);
>     get.addColumn(family, qualifier);
>     get.setMaxVersions();
>     res = this.region.get(get, null);
>     kvs = res.getColumn(family, qualifier);
>     for (int i = 0; i < kvs.size(); i++) {
>       System.out.println(Bytes.toString(kvs.get(i).getValue()));
>     }
>     
>     put = new Put(row);
>     value = Bytes.toBytes("value1");
>     put.add(family, qualifier, 1234567l, value);
>     region.put(put);
>     System.out.print("get value before flush after put value1 : ");
>     get = new Get(row);
>     get.addColumn(family, qualifier);
>     get.setMaxVersions();
>     res = this.region.get(get, null);
>     kvs = res.getColumn(family, qualifier);
>     for (int i = 0; i < kvs.size(); i++) {
>       System.out.println(Bytes.toString(kvs.get(i).getValue()));
>     }
>     region.flushcache();
>     System.out.print("get value after flush after put value1 : ");
>     get = new Get(row);
>     get.addColumn(family, qualifier);
>     get.setMaxVersions();
>     res = this.region.get(get, null);
>     kvs = res.getColumn(family, qualifier);
>     for (int i = 0; i < kvs.size(); i++) {
>       System.out.println(Bytes.toString(kvs.get(i).getValue()));
>     }
>     
>     put = new Put(row);
>     value = Bytes.toBytes("value2");
>     put.add(family, qualifier, 1234567l, value);
>     region.put(put);
>     System.out.print("get value before flush after put value2 : ");
>     get = new Get(row);
>     get.addColumn(family, qualifier);
>     get.setMaxVersions();
>     res = this.region.get(get, null);
>     kvs = res.getColumn(family, qualifier);
>     for (int i = 0; i < kvs.size(); i++) {
>       System.out.println(Bytes.toString(kvs.get(i).getValue()));
>     }
>     region.flushcache();
>     System.out.print("get value after flush after put value2 : ");
>     get = new Get(row);
>     get.addColumn(family, qualifier);
>     get.setMaxVersions();
>     res = this.region.get(get, null);
>     kvs = res.getColumn(family, qualifier);
>     for (int i = 0; i < kvs.size(); i++) {
>       System.out.println(Bytes.toString(kvs.get(i).getValue()));
>     } 
>   }
> {code}
> and the result print as followed:
> {code}
> get value before flush after put value0 : value0
> get value after flush after put value0 : value0
> get value before flush after put value1 : value1
> get value after flush after put value1 : value0
> get value before flush after put value2 : value2
> get value after flush after put value2 : value0
> {code}
> I analyze the code for StoreFileScanner with lazy seek, the StoreFileScanners 
> are sorted by SequenceId, so the latest StoreFile is on the top KeyValueHeap, 
> and the KeyValue for latest StoreFile will comapre to the second latest 
> StoreFile, but the second latest StoreFile generated the fake row for same 
> row, family, qualifier excepts the timestamp( maximum), memstoreTS(0). And 
> the latest KeyValue recognized as not latest than the second latest.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to