[ https://issues.apache.org/jira/browse/HBASE-11882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14118661#comment-14118661 ]
Jerry He commented on HBASE-11882: ---------------------------------- Hi, [~anoop.hbase] bq. In read we set each Cell seqId with the seqId of the file right? Compaction read cells from store files and write to single file. So this set of seqId not happening? Good question! I attached a patch from the testing I was doing. It fixed the problem. But based on your comment, let me confirm if we really need to do the part to add the seqId: {code} + if (current != null && current.isBulkLoaded() + && current.getSequenceID() >= smallestReadPoint) { + kv.setSequenceId(current.getSequenceID()); + } {code} > Row level consistency may not be maintained with bulk load and compaction > ------------------------------------------------------------------------- > > Key: HBASE-11882 > URL: https://issues.apache.org/jira/browse/HBASE-11882 > Project: HBase > Issue Type: Bug > Components: regionserver > Affects Versions: 1.0.0, 2.0.0 > Reporter: Jerry He > Priority: Critical > Fix For: 1.0.0, 2.0.0 > > Attachments: HBASE-11882-master-v1.patch, > TestHRegionServerBulkLoad.java.patch > > > While looking into the TestHRegionServerBulkLoad failure for HBASE-11772, I > found the root cause is that row level atomicity may not be maintained with > bulk load together with compation. > TestHRegionServerBulkLoad is used to test bulk load atomicity. The test uses > multiple threads to do bulk load and scan continuously and do compactions > periodically. > It verifies row level data is always consistent across column families. > After HBASE-11591, we added readpoint checks for bulkloaded data using the > seqId at the time of bulk load. Now a scanner will not see the data from a > bulk load if the scanner's readpoint is earlier than the bulk load seqId. > Previously, the atomic bulk load result is visible immediately to all > scanners. > The problem is with compaction after bulk load. Compaction does not lock the > region and it is done one store (column family) at a time. It also compact > away the seqId marker of bulk load. > Here is an event sequence where the row level consistency is broken. > 1. A scanner is started to scan a region with cf1 and cf2. The readpoint is > 10. > 2. There is a bulk load that loads into cf1 and cf2. The bulk load seqId is > 11. Bulk load is guarded by region write lock. So it is atomic. > 3. There is a compaction that compacts cf1. It compacts away the seqId marker > of the bulk load. > 4. The scanner tries to next to row-1001. It gets the bulk load data for cf1 > since there is no seqId preventing it. It does not get the bulk load data > for cf2 since the scanner's readpoint (10) is less than the bulk load seqId > (11). > Now the row level consistency is broken in this case. -- This message was sent by Atlassian JIRA (v6.3.4#6332)