[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16732500#comment-16732500 ] Josh Elser commented on HBASE-19863: {quote} Looks back on this... Isn't it cheaper to just detect the fake key the same way that ScanQueryMatcher does it - by checking the timestamp for HConstants.OLDEST_TIMESTAMP, instead of doing the compare? If the last K/V is fake we seek, no need to compare the bytes (which is expensive). {quote} Digging back into the archives, [~lhofhansl]. I'm inclined to agree with you at a glance, but I remember being a bit confused by this one the first time around. Any chance I could convince you to throw up a patch in a new Jira issue and see how the test that Sergey added does with the change? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Fix For: 1.5.0, 2.0.0-beta-2, 1.4.3, 2.0.0 > > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.4.patch, > HBASE-19863.v5-branch-1.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16720909#comment-16720909 ] Lars Hofhansl commented on HBASE-19863: --- Looks back on this... Isn't it cheaper to just detect the fake key the same way that ScanQueryMatcher does it - by checking the timestamp for HConstants.OLDEST_TIMESTAMP, instead of doing the compare? If the last K/V is fake we seek, no need to compare the bytes (which is expensive). [~elserj] [~ram_krish] [~anoop.hbase] [~sergey.soldatov] > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Fix For: 1.5.0, 2.0.0-beta-2, 1.4.3, 2.0.0 > > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.4.patch, > HBASE-19863.v5-branch-1.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real"
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16381333#comment-16381333 ] Hudson commented on HBASE-19863: FAILURE: Integrated in Jenkins build HBase-Trunk_matrix #4667 (See [https://builds.apache.org/job/HBase-Trunk_matrix/4667/]) HBASE-19863 java.lang.IllegalStateException: isDelete failed when (elserj: rev 393ab302ab08b70a839ec87e75fcf4b165765db2) * (edit) hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java * (add) hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java * (edit) hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Fix For: 3.0.0, 1.5.0, 2.0.0-beta-2, 1.4.3 > > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.4.patch, > HBASE-19863.v5-branch-1.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16380802#comment-16380802 ] Josh Elser commented on HBASE-19863: {quote} Was: a method that had *hcd.setBloomFilterType(BloomType.NONE);* Now: a new method that has *hcd.setBloomFilterType(type);* with an additional parameter for bloom type and *this* method we are calling from our test. And the old method is just calling the new one with BloomType.NONE as the parameter. So, there are no changes in the behavior. {quote} Yup, I get you. Wasn't saying something was wrong with your patch, just that it looked like the code was inconsistent elsewhere. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.4.patch, > HBASE-19863.v5-branch-1.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16380788#comment-16380788 ] Sergey Soldatov commented on HBASE-19863: - [~elserj] well, my changes are quite simple. Was: a method that had *hcd.setBloomFilterType(BloomType.NONE);* Now: a new method that has *hcd.setBloomFilterType(type);* with an additional parameter for bloom type and *this* method we are calling from our test. And the old method is just calling the new one with BloomType.NONE as the parameter. So, there are no changes in the behavior. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.4.patch, > HBASE-19863.v5-branch-1.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16380738#comment-16380738 ] Josh Elser commented on HBASE-19863: It does look like (at least in branch-1.4) that not all {{createTable}} calls have this logic: {code:java} 1453 public HTable createTable(TableName tableName, byte[][] families, 1454 int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException{code} This signature is missing the {{setBloomFilterType(NONE)}}. I'd say get this in and make a second pass over the branches to make sure all of the createTable methods do the right thing. Let me land this for thanks to the folks who did more review legwork. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.4.patch, > HBASE-19863.v5-branch-1.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16380728#comment-16380728 ] Sergey Soldatov commented on HBASE-19863: - [~ram_krish] actually we were setting it to none by default : {noformat} - // Disable blooms (they are on by default as of 0.95) but we disable them here because - // tests have hard coded counts of what to expect in block cache, etc., and blooms being - // on is interfering. - builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family) - .setBloomFilterType(BloomType.NONE) - .build()); {noformat} That's the part of the code that this method had previously. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.4.patch, > HBASE-19863.v5-branch-1.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16378283#comment-16378283 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 20m 34s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} branch-1.4 Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 8m 0s{color} | {color:green} branch-1.4 passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 40s{color} | {color:green} branch-1.4 passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 37s{color} | {color:green} branch-1.4 passed with JDK v1.7.0_171 {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 27s{color} | {color:green} branch-1.4 passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 12s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 33s{color} | {color:green} branch-1.4 passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 38s{color} | {color:green} branch-1.4 passed with JDK v1.7.0_171 {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 1m 38s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 36s{color} | {color:green} the patch passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 36s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 38s{color} | {color:green} the patch passed with JDK v1.7.0_171 {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 38s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 24s{color} | {color:green} hbase-server: The patch generated 0 new + 384 unchanged - 6 fixed = 384 total (was 390) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 2m 39s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 9m 8s{color} | {color:green} Patch does not cause any errors with Hadoop 2.4.1 2.5.2 2.6.5 2.7.4. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 28s{color} | {color:green} the patch passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 37s{color} | {color:green} the patch passed with JDK v1.7.0_171 {color} | || || || || {color:brown} Other Tests {color} || | {color:red}-1{color} | {color:red} unit {color} | {color:red}135m 47s{color} | {color:red} hbase-server in the patch failed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 32s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}186m 20s{color} | {color:black} {color} | \\ \\ || Reason || Tests || | Failed junit tests | hadoop.hbase.mapreduce.TestSecureLoadIncrementalHFiles | | | hadoop.hbase.mapreduce.TestLoadIncrementalHFilesUseSecurityEndPoint | | | hadoop.hbase.mapreduce.TestLoadIncrementalHFiles | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:74e3133 | | JIRA Issue | HBASE-19863 | | JIRA Patch URL |
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16378031#comment-16378031 ] ramkrishna.s.vasudevan commented on HBASE-19863: LGTM. Just a question {code} public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys, 1389 Configuration c) throws IOException { 1389 Configuration c) throws IOException { {code} This createTable which internally disables Bloom - By default I think we have ROW Bloom enabled. So if this createTable() is used we will be disabling the Blooms always. Is that wanted? Rest looks good to me. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.patch, > HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16378023#comment-16378023 ] Ted Yu commented on HBASE-19863: [~ram_krish]: Do you want to take another look ? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-1.patch, > HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16378015#comment-16378015 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 21s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 1s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} branch-1 Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 2m 1s{color} | {color:green} branch-1 passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 40s{color} | {color:green} branch-1 passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 41s{color} | {color:green} branch-1 passed with JDK v1.7.0_171 {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 28s{color} | {color:green} branch-1 passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 15s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 29s{color} | {color:green} branch-1 passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 38s{color} | {color:green} branch-1 passed with JDK v1.7.0_171 {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 1m 42s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 39s{color} | {color:green} the patch passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 39s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 42s{color} | {color:green} the patch passed with JDK v1.7.0_171 {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 42s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 31s{color} | {color:green} hbase-server: The patch generated 0 new + 384 unchanged - 6 fixed = 384 total (was 390) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 2m 50s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:red}-1{color} | {color:red} hadoopcheck {color} | {color:red} 3m 46s{color} | {color:red} The patch causes 44 errors with Hadoop v2.4.1. {color} | | {color:red}-1{color} | {color:red} hadoopcheck {color} | {color:red} 4m 43s{color} | {color:red} The patch causes 44 errors with Hadoop v2.5.2. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 33s{color} | {color:green} the patch passed with JDK v1.8.0_162 {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 40s{color} | {color:green} the patch passed with JDK v1.7.0_171 {color} | || || || || {color:brown} Other Tests {color} || | {color:red}-1{color} | {color:red} unit {color} | {color:red}107m 12s{color} | {color:red} hbase-server in the patch failed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 18s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}130m 45s{color} | {color:black} {color} | \\ \\ || Reason || Tests || | Failed junit tests | hadoop.hbase.replication.regionserver.TestGlobalThrottler | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:36a7029 | | JIRA Issue | HBASE-19863 | | JIRA Patch URL |
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16377738#comment-16377738 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 23s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 1s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} branch-2 Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 13s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 44s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 10s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 5m 12s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:red}-1{color} | {color:red} findbugs {color} | {color:red} 1m 53s{color} | {color:red} hbase-server in branch-2 has 21 extant Findbugs warnings. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 27s{color} | {color:green} branch-2 passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 24s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 41s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 41s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 8s{color} | {color:green} hbase-server: The patch generated 0 new + 283 unchanged - 6 fixed = 283 total (was 289) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 3m 56s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 14m 24s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. {color} | | {color:green}+1{color} | {color:green} findbugs {color} | {color:green} 2m 1s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 26s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:green}+1{color} | {color:green} unit {color} | {color:green}125m 46s{color} | {color:green} hbase-server in the patch passed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 20s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}161m 21s{color} | {color:black} {color} | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:9f2f2db | | JIRA Issue | HBASE-19863 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12912112/HBASE-19863.v5-branch-2.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux f9f7f9c8eed0 3.13.0-137-generic #186-Ubuntu SMP Mon Dec 4 19:09:19 UTC 2017 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh | | git revision | branch-2 / a312705dbc | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | findbugs | v3.1.0-RC3 | | findbugs | https://builds.apache.org/job/PreCommit-HBASE-Build/11687/artifact/patchprocess/branch-findbugs-hbase-server-warnings.html | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/11687/testReport/ | | Max. process+thread count | 4443 (vs. ulimit of 1) | | modules
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16377500#comment-16377500 ] Sergey Soldatov commented on HBASE-19863: - [~yuzhih...@gmail.com] addressed your comments in v5. [~chia7712] Sure will do it today. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch, HBASE-19863.v5-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16376447#comment-16376447 ] Chia-Ping Tsai commented on HBASE-19863: [~sergey.soldatov] Would you please prepare the patch for branch-1? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16375290#comment-16375290 ] Chia-Ping Tsai commented on HBASE-19863: +1 > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { > Cell
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16375278#comment-16375278 ] Ted Yu commented on HBASE-19863: [~chia7712] [~ram_krish]: Mind taking another look ? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell =
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16375186#comment-16375186 ] Hadoop QA commented on HBASE-19863: --- | (/) *{color:green}+1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 16s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} branch-2 Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 21s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 42s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 8s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 5m 13s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 28s{color} | {color:green} branch-2 passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 24s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 43s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 43s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 9s{color} | {color:green} hbase-server: The patch generated 0 new + 283 unchanged - 6 fixed = 283 total (was 289) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 3s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 14m 35s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 30s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:green}+1{color} | {color:green} unit {color} | {color:green}113m 14s{color} | {color:green} hbase-server in the patch passed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 23s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}144m 12s{color} | {color:black} {color} | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:9f2f2db | | JIRA Issue | HBASE-19863 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12911819/HBASE-19863.v4-branch-2.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux 87ae32597196 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh | | git revision | branch-2 / 4ddfdaffdc | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/11641/testReport/ | | Max. process+thread count | 4851 (vs. ulimit of 1) | | modules | C: hbase-server U: hbase-server | | Console output | https://builds.apache.org/job/PreCommit-HBASE-Build/11641/console | | Powered by | Apache Yetus 0.7.0 http://yetus.apache.org | This message was automatically generated. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16375109#comment-16375109 ] Ted Yu commented on HBASE-19863: Looks good overall. {code} + @BeforeClass public static void setUpBeforeClass() throws Exception { {code} Tradition is to put the annotation on separate line (a few other places). {code} +Table ht = TEST_UTIL {code} Please close the table at the end of the test. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16375057#comment-16375057 ] Sergey Soldatov commented on HBASE-19863: - [~yuzhih...@gmail.com] Ough. Didn't notice that. Reattached. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException {
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16375027#comment-16375027 ] Ted Yu commented on HBASE-19863: The error was on branch-2 v3, not v4. You can re-attach v4 patch one more time. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16375017#comment-16375017 ] Sergey Soldatov commented on HBASE-19863: - That's exactly what I have in the patch {noformat} +@Category({ RegionServerTests.class, FilterTests.class, MediumTests.class }) +public class TestIsDeleteFailure { + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestIsDeleteFailure.class); + {noformat} Both patches for branch-2 and master are the same, but this failure happen only on branch-2 and not reproducible locally. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16374994#comment-16374994 ] Ted Yu commented on HBASE-19863: Please include snippet similar to the following: {code} @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestBackupBoundaryTests.class); {code} > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16374898#comment-16374898 ] Sergey Soldatov commented on HBASE-19863: - [~yuzhih...@gmail.com] that looks like a flaky test and failure is not related to the patch. Passed locally and as we can see passed for branch-2 precommit test. I more worry about the failure for branch-2 patch. Don't understand where that assert error comes from {noformat} java.lang.AssertionError: No HBaseClassTestRule ClassRule for org.apache.hadoop.hbase.regionserver.TestIsDeleteFailure {noformat} > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16374800#comment-16374800 ] Ted Yu commented on HBASE-19863: >From the above QA report: {code} [ERROR] test(org.apache.hadoop.hbase.client.TestAsyncTableGetMultiThreaded) Time elapsed: 139.639 s <<< ERROR! java.util.concurrent.ExecutionException: java.util.concurrent.ExecutionException: org.apache.hadoop.hbase.client.NoServerForRegionException: No server address listed for region 'async,555,1519291655095.5e64d43f832335f627862d2140972538.', row='555', locateType=CURRENT at org.apache.hadoop.hbase.client.TestAsyncTableGetMultiThreaded.test(TestAsyncTableGetMultiThreaded.java:158) Caused by: java.util.concurrent.ExecutionException: org.apache.hadoop.hbase.client.NoServerForRegionException: No server address listed for region 'async,555,1519291655095.5e64d43f832335f627862d2140972538.', row='555', locateType=CURRENT at org.apache.hadoop.hbase.client.TestAsyncTableGetMultiThreaded.run(TestAsyncTableGetMultiThreaded.java:121) at org.apache.hadoop.hbase.client.TestAsyncTableGetMultiThreaded.lambda$null$1(TestAsyncTableGetMultiThreaded.java:134) Caused by: org.apache.hadoop.hbase.client.NoServerForRegionException: No server address listed for region 'async,555,1519291655095.5e64d43f832335f627862d2140972538.', row='555', locateType=CURRENT {code} > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch, HBASE-19863.v4-branch-2.patch, > HBASE-19863.v4-master.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16372607#comment-16372607 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 13s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} master Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 20s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 42s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 8s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 5m 51s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 28s{color} | {color:green} master passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 23s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 42s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 42s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 9s{color} | {color:green} hbase-server: The patch generated 0 new + 283 unchanged - 6 fixed = 283 total (was 289) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 38s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 18m 34s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 27s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:red}-1{color} | {color:red} unit {color} | {color:red}106m 23s{color} | {color:red} hbase-server in the patch failed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 21s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}143m 45s{color} | {color:black} {color} | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:eee3b01 | | JIRA Issue | HBASE-19863 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12911510/HBASE-19863.v4-master.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux 259071e4aeaf 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh | | git revision | master / 2440f807bf | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | unit | https://builds.apache.org/job/PreCommit-HBASE-Build/11614/artifact/patchprocess/patch-unit-hbase-server.txt | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/11614/testReport/ | | Max. process+thread count | 4924 (vs. ulimit of 1) | | modules | C: hbase-server U: hbase-server | | Console output | https://builds.apache.org/job/PreCommit-HBASE-Build/11614/console | | Powered by | Apache Yetus 0.7.0 http://yetus.apache.org | This message was
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16367500#comment-16367500 ] Chia-Ping Tsai commented on HBASE-19863: bq. So that kind of changes may cause a performance degradation for some cases? yep, it introduce the extra disk seek +1 to your solution. Could you add fat comment to say why we need the check? And please add the time rule to your test. {code} +if (nextCell != null && matcher.compareKeyForNextColumn(nextCell, cell) < 0) { + return false; +} {code} > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16366819#comment-16366819 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 17s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} branch-2 Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 43s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 44s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 9s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 5m 20s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 31s{color} | {color:green} branch-2 passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 33s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 45s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 45s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 9s{color} | {color:green} hbase-server: The patch generated 0 new + 283 unchanged - 6 fixed = 283 total (was 289) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 8s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 15m 6s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 29s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:red}-1{color} | {color:red} unit {color} | {color:red}122m 11s{color} | {color:red} hbase-server in the patch failed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 21s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}154m 23s{color} | {color:black} {color} | \\ \\ || Reason || Tests || | Failed junit tests | hadoop.hbase.regionserver.TestIsDeleteFailure | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:9f2f2db | | JIRA Issue | HBASE-19863 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12910881/HBASE-19863.v3-branch-2.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux 1dd34530213d 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build@2/component/dev-support/hbase-personality.sh | | git revision | branch-2 / 9f27fdaffc | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | unit | https://builds.apache.org/job/PreCommit-HBASE-Build/11549/artifact/patchprocess/patch-unit-hbase-server.txt | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/11549/testReport/ | | Max. process+thread count | 4936 (vs. ulimit of 1) | | modules | C: hbase-server U: hbase-server | | Console output |
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16366693#comment-16366693 ] Sergey Soldatov commented on HBASE-19863: - [~chia7712] I've moved the test to a separate file. I don't think it affects trySkipToNextRow. This problem happens inside a single row only, so even if we jump back when getting a new top, we will just iterate using next() to the next row anyway... As for changing realSeekDone to false, I was thinking that it was done especially to avoid some seeks (not only the case we are discussing). So that kind of changes may cause a performance degradation for some cases? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch, > HBASE-19863.v3-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16365559#comment-16365559 ] Chia-Ping Tsai commented on HBASE-19863: BTW, the test is not in the appropriate place. Move it to a separate file? Does the bug happen in trySkipToNextRow also? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do {
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16364898#comment-16364898 ] Chia-Ping Tsai commented on HBASE-19863: The trouble is caused by that the current kv scanner in kv heap doesn’t do the real seek. The KeyValueHeap#pollRealKV do the check through KeyValueScanner#realSeekDone, hence it assumes the cell from current kv scanner is located at or after the seek cell. StoreFileScanner doesn’t tell the true so the cell returned by StoreFileScanner may be located before the seek key. v2 patch is fine to me. Although, it may be better to do the following change to ensure kv heap always does the seek when updating current kv scanner. {code:title=StoreFileScanner.java} setCurrentCell(PrivateCellUtil.createLastOnRowCol(kv)); realSeekDone = true; // we should make it false {code} A side-effect, however, is that one more disk seek is generated when store scanner fails to collect enough column-value cells, since kv heap will enforce the last (lazy)file scanner to do the real seek… > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16364544#comment-16364544 ] Chia-Ping Tsai commented on HBASE-19863: {quote}[~chia7712] - you have any comments? {quote} Working at comprehending the root causeWill come back asap. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { >
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16364201#comment-16364201 ] ramkrishna.s.vasudevan commented on HBASE-19863: Patch looks good to me. If the test is taking time to run probably you can avoid the spinning of cluster and just create Hregion and add data to the region directly. Else +1. Lets wait for other reviews if you want to get it in. [~chia7712] - you have any comments? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn >
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16363871#comment-16363871 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 17s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} branch-2 Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 53s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 54s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 20s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 6m 10s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 32s{color} | {color:green} branch-2 passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 4s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 48s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 48s{color} | {color:green} the patch passed {color} | | {color:red}-1{color} | {color:red} checkstyle {color} | {color:red} 1m 19s{color} | {color:red} hbase-server: The patch generated 2 new + 324 unchanged - 3 fixed = 326 total (was 327) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 29s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 17m 4s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 35s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:red}-1{color} | {color:red} unit {color} | {color:red}113m 1s{color} | {color:red} hbase-server in the patch failed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 20s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}150m 41s{color} | {color:black} {color} | \\ \\ || Reason || Tests || | Failed junit tests | hadoop.hbase.client.TestAsyncTable | | | hadoop.hbase.master.TestDLSFSHLog | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:9f2f2db | | JIRA Issue | HBASE-19863 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12910558/HBASE-19863.v2-branch-2.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux d61b9e2a58cf 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh | | git revision | branch-2 / 02bba3b788 | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | checkstyle | https://builds.apache.org/job/PreCommit-HBASE-Build/11521/artifact/patchprocess/diff-checkstyle-hbase-server.txt | | unit | https://builds.apache.org/job/PreCommit-HBASE-Build/11521/artifact/patchprocess/patch-unit-hbase-server.txt | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/11521/testReport/ | | Max.
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16363718#comment-16363718 ] Sergey Soldatov commented on HBASE-19863: - [~yuzhih...@gmail.com] you mean checkstyle? Addressed that in v2 patch. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch, HBASE-19863.v2-branch-2.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { > Cell nextIndexedKey =
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16363698#comment-16363698 ] Ted Yu commented on HBASE-19863: Lgtm Sergey: Can you check findbugs warnings ? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch-2.patch, HBASE-19863-branch1.patch, > HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { > Cell nextIndexedKey = getNextIndexedKey(); > if (nextIndexedKey != null && nextIndexedKey != >
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16363671#comment-16363671 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 17s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 2 new or modified test files. {color} | || || || || {color:brown} branch-2 Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 37s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 48s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 12s{color} | {color:green} branch-2 passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 5m 27s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 27s{color} | {color:green} branch-2 passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 22s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 42s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 42s{color} | {color:green} the patch passed {color} | | {color:red}-1{color} | {color:red} checkstyle {color} | {color:red} 1m 10s{color} | {color:red} hbase-server: The patch generated 7 new + 327 unchanged - 0 fixed = 334 total (was 327) {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 15s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 15m 23s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 29s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:green}+1{color} | {color:green} unit {color} | {color:green}106m 3s{color} | {color:green} hbase-server in the patch passed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 20s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}138m 19s{color} | {color:black} {color} | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:9f2f2db | | JIRA Issue | HBASE-19863 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12910523/HBASE-19863-branch-2.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux 0561738b1214 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh | | git revision | branch-2 / 02bba3b788 | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | checkstyle | https://builds.apache.org/job/PreCommit-HBASE-Build/11518/artifact/patchprocess/diff-checkstyle-hbase-server.txt | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/11518/testReport/ | | Max. process+thread count | 4823 (vs. ulimit of 1) | | modules | C: hbase-server U: hbase-server | | Console output | https://builds.apache.org/job/PreCommit-HBASE-Build/11518/console | | Powered by | Apache Yetus 0.7.0 http://yetus.apache.org | This
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16361916#comment-16361916 ] Sergey Soldatov commented on HBASE-19863: - [~ram_krish] thank you, sir! I will prepare the final patch tomorrow (it would require some changes in test harness to make bloom filter configurable). > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { > Cell
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16361906#comment-16361906 ] ramkrishna.s.vasudevan commented on HBASE-19863: I think this patch is fine. We are seeing a scanner lags behind and we are forcing a seek here. In my opinion we can avoid the looping also as I said in earlier comment but if that seems risky or if we feel it could affect other cases then this fix is fine. We are doing a seek probably in the same block only here (correct me if am wrong) but with the help of the column tracker. So not in this patch - I think internally if we try to do a seek on the same block we actually don fetch the block again I believe. So it should not be costly too. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16358803#comment-16358803 ] Josh Elser commented on HBASE-19863: [~ram_krish], [~anoop.hbase], seems to me that you guys understand the problem much better than I do. Any chance I could ask you to keep up on the reviews? I am worried that I don't yet understand the problem well enough to give a proper review alone. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16355179#comment-16355179 ] Anoop Sam John commented on HBASE-19863: Ya understood the real issue. We need to make sure the scanner is really reached to the position where it is supposed to be. (In reality it is lagging as we use some fake keys).. I thought here the later discussion was around, for this jump to the actual needed pos, we should do one seek or many skips. Not just one skip. Was just saying the old design. I did not check this patch yet. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16355021#comment-16355021 ] Sergey Soldatov commented on HBASE-19863: - [~anoop.hbase] thank you, sir! The question is whether it's ok to do seek if during the heap.next() we choose another scanner which is actually 'lag' behind (for example cur points to C10, but hfs is still pointing to C1). If we try to skip here we may break the order (so, at the moment we are at C10, but during the next instead of C11 we will get C2. I tried to find a better place to fix it but didn't find any. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. >
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16354998#comment-16354998 ] Anoop Sam John commented on HBASE-19863: Yes that jira did this optimization. When we have to reach to a key (by skip to next col or row) and that key is going to be within the same block, we would like to do skips not seek. When that jump can take us to next block, the seek is better. This is what the optimization doing. So this same design we have to keep at any cases. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn >
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16352674#comment-16352674 ] Sergey Soldatov commented on HBASE-19863: - bq. I agree to this. But my arg would be that if getNextIndexedKey() is a key with the next row It's usually a key with the next row. And in this case, the (1) check just return the value greater than zero, so we will perform next() once. After that, in (2) we just check that we are still in the same column and if not we return true, avoiding the call of reseek. bq. Expectation is for compareKeyForNextColumn() check with nextcolumn. According to the method name it's true, but according to the comments this is just a "result of the compare between the indexed key and the key portion of the passed cell". I believe that all this logic fits [~lhofhansl] optimization for SEEK vs SKIP that is implemented in HBASE-13109 > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16352088#comment-16352088 ] ramkrishna.s.vasudevan commented on HBASE-19863: bq.So the reason for this check is only to check that we are still in the same block, so no need for reseek I agree to this. But my arg would be that if getNextIndexedKey() is a key with the next row but we are actually trying to do only skipToNextCol then lets simply go with the reseek way? Doing next on the current block is not going to help us . Also see the method name compareKeyForNextColumn(). we also have compareKeyForNextRow(). Expectation is for compareKeyForNextColumn() check with nextcolumn. But internally our compare method is such that we first check row and if that row is bigger than the current cell we still think we have found the next index. Just my 2 cents. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column.
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16345868#comment-16345868 ] Sergey Soldatov commented on HBASE-19863: - [~ram_krish] That's an interesting question what trySkipToNextColumn is supposed to do. If I understand correctly it's just tries to really skip to the next column in the row without reseek. There are two checks: 1. {noformat} matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0 {noformat} So the reason of this check is only to check that we are still in the same block, so no need for reseek 2. {noformat} CellUtil.matchingRowColumn(cell, nextCell) {noformat} That would work when we iterate through versions of the same cell. Based on those two conditions we decide whether we need to reseek (at the moment only if we reached the block end or there is no indexed keys at all), or just continue to iterate in StoreScanner.next with new cell. So, my fix is just handle the case when heap.next() change the top scanner which lags behind, so second check thinks that we on the next cell, but actually we just jumped back and need to perform reseek. And yes, the original description is just a corner case of the wider problem when we change our top scanner to the one which skipped last reseek due optimization with fake KVs. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16345642#comment-16345642 ] Josh Elser commented on HBASE-19863: Cool, thanks for verifying [~ram_krish]. I had just assumed they were equivalent, but thought it good to make sure :) Working with Sergey on a $dayjob escalation, but I imagine he'll get a new patch up later today. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16345389#comment-16345389 ] ramkrishna.s.vasudevan commented on HBASE-19863: IMO both cases (the description here and the exception that the test case here creates) and the reason are the same. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { > Cell nextIndexedKey =
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16345386#comment-16345386 ] Ted Yu commented on HBASE-19863: I was expecting exception to come out of ScanDeleteTracker (shown in description). I am not sure whether the two exceptions reflect the same root-cause. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { > Cell nextIndexedKey =
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16345370#comment-16345370 ] Josh Elser commented on HBASE-19863: bq. With the attached test case the issue that we get is Wrong scan order and not the description as in this JIRA here [~yuzhih...@gmail.com], do you mean to imply that the failure that your test case shows is the same root-cause that Sergey is trying to fix? In other words, the exception the test throws and the exception in the description are two manifestations of the same problem? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16344814#comment-16344814 ] ramkrishna.s.vasudevan commented on HBASE-19863: With the attached test case the issue that we get is Wrong scan order and not the description as in this JIRA here. But thanks to [~sergey.soldatov] for this case. Your fix makes sense according to me. Now suppose in row key1000 we have c2 to c15. Ideally when we are at c3 and the column tracker says skip to next col we should only skip to the col that the col tracker says. We try to do that based on the nextIndexedKey() but this nextIndexedKey is already at the next row ie. key1. So we simply keep checking and once we get the next col in the current row key1000 we move to that. So from c3 we move to c4 by not respecting what the ExplicitColumn tracker says. Though the fix is correct I think we can even avoid some iterations before deciding on to compareKeyForNextColumn(nextCell, cell). Like if the nextIndexedKey is already at the nextRow just return false. And make the {code} seekAsDirection(matcher.getKeyForNextColumn(cell)); {code} call? Because when the tracker says c9 it is better we only take c9 instead of going from c3, c4, c45, c7, c8 and then to c9. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-branch1.patch, HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16344426#comment-16344426 ] Hadoop QA commented on HBASE-19863: --- | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 11s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} hbaseanti {color} | {color:green} 0m 0s{color} | {color:green} Patch does not have any anti-patterns. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:red}-1{color} | {color:red} test4tests {color} | {color:red} 0m 0s{color} | {color:red} The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. {color} | || || || || {color:brown} master Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 59s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 48s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 13s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 6m 6s{color} | {color:green} branch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 28s{color} | {color:green} master passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 4m 21s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 43s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 43s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 1m 5s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedjars {color} | {color:green} 4m 46s{color} | {color:green} patch has no errors when building our shaded downstream artifacts. {color} | | {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 18m 25s{color} | {color:green} Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 28s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:red}-1{color} | {color:red} unit {color} | {color:red}105m 41s{color} | {color:red} hbase-server in the patch failed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 19s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black}143m 49s{color} | {color:black} {color} | \\ \\ || Reason || Tests || | Failed junit tests | hadoop.hbase.TestZooKeeper | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:eee3b01 | | JIRA Issue | HBASE-19863 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12908234/HBASE-19863-branch1.patch | | Optional Tests | asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile | | uname | Linux a53b47ac1993 3.13.0-133-generic #182-Ubuntu SMP Tue Sep 19 15:49:21 UTC 2017 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh | | git revision | master / 34c6c99041 | | maven | version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) | | Default Java | 1.8.0_151 | | unit | https://builds.apache.org/job/PreCommit-HBASE-Build/11240/artifact/patchprocess/patch-unit-hbase-server.txt | | Test Results | https://builds.apache.org/job/PreCommit-HBASE-Build/11240/testReport/ | | modules | C: hbase-server U: hbase-server | | Console output | https://builds.apache.org/job/PreCommit-HBASE-Build/11240/console | | Powered by |
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16343845#comment-16343845 ] Sergey Soldatov commented on HBASE-19863: - bq. In this place we try to do a real seek right before actually using that scanner? I remember this to happen if we are in a fake key. There are some exceptions. StoreScanner.trySkipToNextColumn as an example. If we have indexed keys, we just iterate through the heap without reseek (which may happen in seekOrSkipToNextColumn if trySkipToNextColumn return false). > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16343829#comment-16343829 ] Ted Yu commented on HBASE-19863: The test patch was based on branch-1. When I ran TestAdmin1#testIsDeleteFailure : {code} Mon Jan 29 19:01:06 UTC 2018, RpcRetryingCaller{globalStartTime=1517252466261, pause=250, retries=6}, java.io.IOException: java.io.IOException: Key key10018/0:C09/OLDEST_TIMESTAMP/Minimum/vlen=0/seqid=0 followed by a smaller key key10018/0:C04/1517252461198/Put/vlen=3/seqid=4 in cf 0 at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2412) at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:124) at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:297) at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:277) Caused by: java.lang.AssertionError: Key key10018/0:C09/OLDEST_TIMESTAMP/Minimum/vlen=0/seqid=0 followed by a smaller key key10018/0:C04/1517252461198/Put/vlen=3/seqid=4 in cf 0 at org.apache.hadoop.hbase.regionserver.StoreScanner.checkScanOrder(StoreScanner.java:975) at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:555) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:152) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:6219) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6379) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:6151) at org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2808) at org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:3045) at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:36613) at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2354) ... 3 more {code} The above was not exactly the same as the exception shown in the description. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16343811#comment-16343811 ] Sergey Soldatov commented on HBASE-19863: - Attached a test case to reproduce the problem for branch-1. It has some changes in HBaseTestingUtility.createTable to make required conditions (bloom filter as well as block size). Please let me know whether it helps to reproduce the problem. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > Attachments: HBASE-19863-test.patch > > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException {
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16342999#comment-16342999 ] ramkrishna.s.vasudevan commented on HBASE-19863: Ok. So I had that doubt but sine it was talking about stores A and B I wanted to clarify. bq.KeyValueHeap.next() knows nothing about fake rows so at one moment it may use scanner that has inconsistent state (cur value is far behind of hfs.keyValue, so seek is required before next()) In this place we try to do a real seek right before actually using that scanner? I remember this to happen if we are in a fake key. Steps to reproduce will help to debug and come up with a solution. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16342971#comment-16342971 ] Anoop Sam John commented on HBASE-19863: bq.Two HStore A and B? It is actually one single column family 0 right? So how come two HStores. I think its a typo Ram. It is single CF and 2 store files. The below explanation also says there is 2 files and so 2 Store file scanners. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell = null; > do { > Cell
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16342970#comment-16342970 ] Anoop Sam John commented on HBASE-19863: bq.So, in next() when we set new current scanner we may check whether it has cur set to a fake value and enforce seek if so Ya I think we need this to be done.. When we have to do seeks and we see no such cells in this HFIle, we make the fake cells. The other HFile scanners will get the real cells. But we have to do a real seek before doing any further ops on the previous HFile's scanners. I think we were doing those before. I remember reading such code in old days. Not sure where and how.. It would have been best if we have a test case for simulating this. Any luck for doing that? Good one. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. >
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16342963#comment-16342963 ] ramkrishna.s.vasudevan commented on HBASE-19863: bq.The table has a single region with two HStore: Two HStore A and B? It is actually one single column family 0 right? So how come two HStores. Per Column family we have one HStore. Ideally if we have explicitly marked the family and the qualifier the explicit column tracker would ideally keep skipping to the given qualifiers only. So for the two hstores is it something Phoenix adds? bq.The exception was first observed thru Phoenix query but can be reproduced in the given scenario thru shell scan. Can you tell the steps to produce through shell scan? Including the DDL and the put commands? > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher.
[jira] [Commented] (HBASE-19863) java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used
[ https://issues.apache.org/jira/browse/HBASE-19863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16340334#comment-16340334 ] Ted Yu commented on HBASE-19863: This case first came up during the past weekend (I was on call). The exception was first observed thru Phoenix query but can be reproduced in the given scenario thru shell scan. The current workaround is to disable ROWCOL bloom for the column family but this reduces the scan performance. > java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter > is used > - > > Key: HBASE-19863 > URL: https://issues.apache.org/jira/browse/HBASE-19863 > Project: HBase > Issue Type: Bug > Components: Filters >Affects Versions: 1.4.1 >Reporter: Sergey Soldatov >Assignee: Sergey Soldatov >Priority: Major > > Under some circumstances scan with SingleColumnValueFilter may fail with an > exception > {noformat} > java.lang.IllegalStateException: isDelete failed: deleteBuffer=C3, > qualifier=C2, timestamp=1516433595543, comparison result: 1 > at > org.apache.hadoop.hbase.regionserver.ScanDeleteTracker.isDeleted(ScanDeleteTracker.java:149) > at > org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:386) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:545) > at > org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5876) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6027) > at > org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5814) > at > org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2552) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32385) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:112) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:187) > at > org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:167) > {noformat} > Conditions: > table T with a single column family 0 that uses ROWCOL bloom filter > (important) and column qualifiers C1,C2,C3,C4,C5. > When we fill the table for every row we put deleted cell for C3. > The table has a single region with two HStore: > A: start row: 0, stop row: 99 > B: start row: 10 stop row: 99 > B has newer versions of rows 10-99. Store files have several blocks each > (important). > Store A is the result of major compaction, so it doesn't have any deleted > cells (important). > So, we are running a scan like: > {noformat} > scan 'T', { COLUMNS => ['0:C3','0:C5'], FILTER => "SingleColumnValueFilter > ('0','C5',=,'binary:whatever')"} > {noformat} > How the scan performs: > First, we iterate A for rows 0 and 1 without any problems. > Next, we start to iterate A for row 10, so read the first cell and set hfs > scanner to A : > 10:0/C1/0/Put/x but found that we have a newer version of the cell in B : > 10:0/C1/1/Put/x, > so we make B as our current store scanner. Since we are looking for > particular columns > C3 and C5, we perform the optimization StoreScanner.seekOrSkipToNextColumn > which > would run reseek for all store scanners. > For store A the following magic would happen in requestSeek: > 1. bloom filter check passesGeneralBloomFilter would set haveToSeek to > false because row 10 doesn't have C3 qualifier in store A. > 2. Since we don't have to seek we just create a fake row > 10:0/C3/OLDEST_TIMESTAMP/Maximum, an optimization that is quite important for > us and it commented with : > {noformat} > // Multi-column Bloom filter optimization. > // Create a fake key/value, so that this scanner only bubbles up to the > top > // of the KeyValueHeap in StoreScanner after we scanned this row/column in > // all other store files. The query matcher will then just skip this fake > // key/value and the store scanner will progress to the next column. This > // is obviously not a "real real" seek, but unlike the fake KV earlier in > // this method, we want this to be propagated to ScanQueryMatcher. > {noformat} > > For store B we would set it to fake 10:0/C3/createFirstOnRowColTS()/Maximum > to skip C3 entirely. > After that we start searching for qualifier C5 using seekOrSkipToNextColumn > which run first trySkipToNextColumn: > {noformat} > protected boolean trySkipToNextColumn(Cell cell) throws IOException { > Cell nextCell =