[ https://issues.apache.org/jira/browse/HBASE-6269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13402684#comment-13402684 ]
Hudson commented on HBASE-6269: ------------------------------- Integrated in HBase-TRUNK-on-Hadoop-2.0.0 #72 (See [https://builds.apache.org/job/HBase-TRUNK-on-Hadoop-2.0.0/72/]) HBASE-6269 Lazyseek should use the maxSequenseId StoreFile's KeyValue as the latest KeyValue (Xing Shi) (Revision 1354703) Result = FAILURE tedyu : Files : * /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java * /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java > 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-trunk-V1.patch, HBASE-6269-v1.patch, > runAllTests.out.txt > > > 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