You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by "chenglei (JIRA)" <ji...@apache.org> on 2017/02/21 09:31:44 UTC

[jira] [Comment Edited] (PHOENIX-3578) Incorrect query results when applying inner join and orderby desc

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

chenglei edited comment on PHOENIX-3578 at 2/21/17 9:30 AM:
------------------------------------------------------------

This issue is caused by the join dynamic filter, from following RHS, we get d.id is in (1,2):
{code:borderStyle=solid} 
      select d.seq,d.col2,d.id from detail d  where d.id between 1 and 2
{code} 
so with join dynamic filter, m.id is also in (1,2). Before applying join dynamic filter,LHS is:
{code:borderStyle=solid}
      select m,id,m.col1,d.seq,d.col2 from master m order by m.id desc
{code}
Obviously, LHS's OrderBy is {{OrderBy.REV_ROW_KEY_ORDER_BY}},after applying join dynamic filter LHS turns to :
{code:borderStyle=solid} 
    select m,id,m.col1,d.seq,d.col2 from master m where m.id in (1,2) order by m.id desc
{code} 
Notice LHS's OrderBy is still {{OrderBy.REV_ROW_KEY_ORDER_BY}} now,then {{WhereOptimizer.pushKeyExpressionsToScan}} was called to push {{m.id in (1,2)}} into Scan , and useSkipScan is true in following line 274 of {{WhereOptimizer.pushKeyExpressionsToScan}} method: 
{code:borderStyle=solid}
273        stopExtracting |= (hasUnboundedRange && !forcedSkipScan) || (hasRangeKey && forcedRangeScan);
274        useSkipScan |= !stopExtracting && !forcedRangeScan && (keyRanges.size() > 1 || hasRangeKey);
{code} 

next step the {{startRow}} and {{endRow}} of LHS's Scan was computed in {{ScanRanges.create}} method, in following line 112 the LHS's RowKeySchema is turned to SchemaUtil.VAR_BINARY_SCHEMA: 

{code:borderStyle=solid} 
111  if (keys.size() > 1 || SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), false, schema.getField(schema.getFieldCount()-1)) == QueryConstants.DESC_SEPARATOR_BYTE) {
112                schema = SchemaUtil.VAR_BINARY_SCHEMA;
113                slotSpan = ScanUtil.SINGLE_COLUMN_SLOT_SPAN;
114           } else { 
{code}

so in following line 135 and line 136 of {{ScanRanges.create}} method,minKey is {{\\x80\\x00\\x00\\x01}},and maxKey is \\x80\\x00\\x00\\x02\\x00, and correspondingly,the Scan's startRow is \\x80\\x00\\x00\\x01, and Scan's endRow is \\x80\\x00\\x00\\x02\\x00:
{code:borderStyle=solid}
134        if (nBuckets == null || !isPointLookup || !useSkipScan) {
135            byte[] minKey = ScanUtil.getMinKey(schema, sortedRanges, slotSpan);
136            byte[] maxKey = ScanUtil.getMaxKey(schema, sortedRanges, slotSpan);
{code}

In summary, when we scan the LHS {{master}} table, the Scan range is {{[\\x80\\x00\\x00\\x01,\\x80\\x00\\x00\\x02\\x00)}} ,and the Scan uses {{SkipScanFilter}}.Furthermore,because the LHS's OrderBy is {{OrderBy.REV_ROW_KEY_ORDER_By}},so the Scan range should be reversed.In {{BaseScannerRegionObserver.preScannerOpen}} method,following {{ScanUtil.setupReverseScan}} method is called to reverse the Scan's startRow and endRow.Unfortunately, the reversed Scan's range computed by  {{ScanUtil.setupReverseScan}} method is [\\x80\\x00\\x00\\x01\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF,\\x80\\x00\\x00\\x00\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF), so we can only get the row of {{master}} table which id is 1, the row which id is 2 is excluded.

{code:borderStyle=solid} 
621  public static void setupReverseScan(Scan scan) {
622        if (isReversed(scan)) {
623            byte[] newStartRow = getReversedRow(scan.getStartRow());
624            byte[] newStopRow = getReversedRow(scan.getStopRow());
625            scan.setStartRow(newStopRow);
626            scan.setStopRow(newStartRow);
627            scan.setReversed(true);
628        }
629    }  
{code}

In conclusion, following two problems causes this issue:
(1) the {{ScanUtil.getReversedRow}} method is not right for {{\\x80\\x00\\x00\\x02\\x00}},which should return {{\\x80\\x00\\x00\\x02}},not {{\\x80\\x00\\x00\\x01\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF}}.
(2) even though {{ScanUtil.getReversedRow}} method is right,there may be another problem,if I change the table data as following :
{code:borderStyle=solid}
            UPSERT INTO master VALUES (1, 'A1');
            UPSERT INTO master VALUES (2, 'A2');
            UPSERT INTO master VALUES (3, 'A3');
            UPSERT INTO master VALUES (4, 'A4');
            UPSERT INTO master VALUES (5, 'A5');
            UPSERT INTO master VALUES (6, 'A6');
            UPSERT INTO master VALUES (8, 'A8');
            
            UPSERT INTO detail VALUES (1, 1, 'B1');
            UPSERT INTO detail VALUES (2, 2, 'B2');
            UPSERT INTO detail VALUES (3, 3, 'B3');
            UPSERT INTO detail VALUES (4, 4, 'B4');
            UPSERT INTO detail VALUES (5, 5, 'B5');
            UPSERT INTO detail VALUES (6, 6, 'B6');
            UPSERT INTO detail VALUES (7, 7, 'B7');
            UPSERT INTO detail VALUES (8, 8, 'B8');
(code}

and modify the sql as :
{code:borderStyle=solid}
   select m.id, m.col1,d.col2 from master m, detail d  where m.id = d.id  and d.id in (3,5,7) order by m.id desc
{code},

because the {{master}} table does not have rows which id=7,so the result of {{ScanUtil.getReversedRow}} method is right,but executing the above sql would throw following Exception(Under HBase 1.2.0 or lower):
{code:borderStyle=solid}
   org.apache.phoenix.exception.PhoenixIOException: org.apache.hadoop.hbase.DoNotRetryIOException: T000006,,1487348132415.92ad11583fe62ed8604f259d555b8f32.: The next hint must come after previous hint (prev=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, next=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, kv=\x80\x00\x00\x06/0:COL1/1487348135903/Put/vlen=2/seqid=4)
        at org.apache.phoenix.util.ServerUtil.createIOException(ServerUtil.java:89)
        at org.apache.phoenix.util.ServerUtil.throwIOException(ServerUtil.java:55)
        at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$1.nextRaw(BaseScannerRegionObserver.java:489)
        at org.apache.phoenix.coprocessor.HashJoinRegionScanner.nextRaw(HashJoinRegionScanner.java:274)
        at org.apache.phoenix.coprocessor.DelegateRegionScanner.nextRaw(DelegateRegionScanner.java:77)
        at org.apache.phoenix.coprocessor.DelegateRegionScanner.nextRaw(DelegateRegionScanner.java:77)
        at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$RegionScannerHolder.nextRaw(BaseScannerRegionObserver.java:261)
        at org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2555)
        at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:33648)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2170)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:109)
        at org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
        at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.IllegalStateException: The next hint must come after previous hint (prev=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, next=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, kv=\x80\x00\x00\x06/0:COL1/1487348135903/Put/vlen=2/seqid=4)
        at org.apache.phoenix.filter.SkipScanFilter.setNextCellHint(SkipScanFilter.java:171)
        at org.apache.phoenix.filter.SkipScanFilter.filterKeyValue(SkipScanFilter.java:145)
        at org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:427)
        at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:522)
        at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147)
        at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5661)
        at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:5820)
        at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5598)
        at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$1.nextRaw(BaseScannerRegionObserver.java:459)
        ... 11 more
{code}

That is because {{SkipScanFilter}} can not support {{OrderBy.REV_ROW_KEY_ORDER_BY}} now, if the OrderBy is  {{OrderBy.REV_ROW_KEY_ORDER_BY}}, we can not use {{SkipScanFilter}} after applying  join dynamic filter.


was (Author: comnetwork):
This issue is caused by the join dynamic filter, from following RHS, we get d.id is in (1,2):
{code:borderStyle=solid} 
      select d.seq,d.col2,d.id from detail d  where d.id between 1 and 2
{code} 
so with join dynamic filter, m.id is also in (1,2). Before applying join dynamic filter,LHS is:
{code:borderStyle=solid}
      select m,id,m.col1,d.seq,d.col2 from master m order by m.id desc
{code}
Obviously, LHS's OrderBy is {{OrderBy.REV_ROW_KEY_ORDER_BY}},after applying join dynamic filter LHS turns to :
{code:borderStyle=solid} 
    select m,id,m.col1,d.seq,d.col2 from master m where m.id in (1,2) order by m.id desc
{code} 
Notice LHS's OrderBy is still {{OrderBy.REV_ROW_KEY_ORDER_BY}} now,then {{WhereOptimizer.pushKeyExpressionsToScan}} was called to push {{m.id in (1,2)}} into Scan , and useSkipScan is true in following line 274 of {{WhereOptimizer.pushKeyExpressionsToScan}} method: 
{code:borderStyle=solid}
273        stopExtracting |= (hasUnboundedRange && !forcedSkipScan) || (hasRangeKey && forcedRangeScan);
274        useSkipScan |= !stopExtracting && !forcedRangeScan && (keyRanges.size() > 1 || hasRangeKey);
{code} 

next step the {{startRow}} and {{endRow}} of LHS's Scan was computed in {{ScanRanges.create}} method, in following line 112 the LHS's RowKeySchema is turned to SchemaUtil.VAR_BINARY_SCHEMA: 

{code:borderStyle=solid} 
111  if (keys.size() > 1 || SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), false, schema.getField(schema.getFieldCount()-1)) == QueryConstants.DESC_SEPARATOR_BYTE) {
112                schema = SchemaUtil.VAR_BINARY_SCHEMA;
113                slotSpan = ScanUtil.SINGLE_COLUMN_SLOT_SPAN;
114           } else { 
{code}

so in following line 135 and line 136 of {{ScanRanges.create}} method,minKey is \\x80\\x00\\x00\\x01,and maxKey is \\x80\\x00\\x00\\x02\\x00, and correspondingly,the Scan's startRow is \\x80\\x00\\x00\\x01, and Scan's endRow is \\x80\\x00\\x00\\x02\\x00:
{code:borderStyle=solid}
134        if (nBuckets == null || !isPointLookup || !useSkipScan) {
135            byte[] minKey = ScanUtil.getMinKey(schema, sortedRanges, slotSpan);
136            byte[] maxKey = ScanUtil.getMaxKey(schema, sortedRanges, slotSpan);
{code}

In summary, when we scan the LHS {{master}} table, the Scan range is {{[\\x80\\x00\\x00\\x01,\\x80\\x00\\x00\\x02\\x00)}} ,and the Scan uses {{SkipScanFilter}}.Furthermore,because the LHS's OrderBy is {{OrderBy.REV_ROW_KEY_ORDER_By}},so the Scan range should be reversed.In {{BaseScannerRegionObserver.preScannerOpen}} method,following {{ScanUtil.setupReverseScan}} method is called to reverse the Scan's startRow and endRow.Unfortunately, the reversed Scan's range computed by  {{ScanUtil.setupReverseScan}} method is [\\x80\\x00\\x00\\x01\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF,\\x80\\x00\\x00\\x00\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF), so we can only get the row of {{master}} table which id is 1, the row which id is 2 is excluded.

{code:borderStyle=solid} 
621  public static void setupReverseScan(Scan scan) {
622        if (isReversed(scan)) {
623            byte[] newStartRow = getReversedRow(scan.getStartRow());
624            byte[] newStopRow = getReversedRow(scan.getStopRow());
625            scan.setStartRow(newStopRow);
626            scan.setStopRow(newStartRow);
627            scan.setReversed(true);
628        }
629    }  
{code}

In conclusion, following two problems causes this issue:
(1) the {{ScanUtil.getReversedRow}} method is not right for {{\\x80\\x00\\x00\\x02\\x00}},which should return {{\\x80\\x00\\x00\\x02}},not {{\\x80\\x00\\x00\\x01\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF}}.
(2) even though {{ScanUtil.getReversedRow}} method is right,there may be another problem,if I change the table data as following :
{code:borderStyle=solid}
            UPSERT INTO master VALUES (1, 'A1');
            UPSERT INTO master VALUES (2, 'A2');
            UPSERT INTO master VALUES (3, 'A3');
            UPSERT INTO master VALUES (4, 'A4');
            UPSERT INTO master VALUES (5, 'A5');
            UPSERT INTO master VALUES (6, 'A6');
            UPSERT INTO master VALUES (8, 'A8');
            
            UPSERT INTO detail VALUES (1, 1, 'B1');
            UPSERT INTO detail VALUES (2, 2, 'B2');
            UPSERT INTO detail VALUES (3, 3, 'B3');
            UPSERT INTO detail VALUES (4, 4, 'B4');
            UPSERT INTO detail VALUES (5, 5, 'B5');
            UPSERT INTO detail VALUES (6, 6, 'B6');
            UPSERT INTO detail VALUES (7, 7, 'B7');
            UPSERT INTO detail VALUES (8, 8, 'B8');
(code}

and modify the sql as :
{code:borderStyle=solid}
   select m.id, m.col1,d.col2 from master m, detail d  where m.id = d.id  and d.id in (3,5,7) order by m.id desc
{code},

because the {{master}} table does not have rows which id=7,so the result of {{ScanUtil.getReversedRow}} method is right,but executing the above sql would throw following Exception(Under HBase 1.2.0 or lower):
{code:borderStyle=solid}
   org.apache.phoenix.exception.PhoenixIOException: org.apache.hadoop.hbase.DoNotRetryIOException: T000006,,1487348132415.92ad11583fe62ed8604f259d555b8f32.: The next hint must come after previous hint (prev=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, next=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, kv=\x80\x00\x00\x06/0:COL1/1487348135903/Put/vlen=2/seqid=4)
        at org.apache.phoenix.util.ServerUtil.createIOException(ServerUtil.java:89)
        at org.apache.phoenix.util.ServerUtil.throwIOException(ServerUtil.java:55)
        at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$1.nextRaw(BaseScannerRegionObserver.java:489)
        at org.apache.phoenix.coprocessor.HashJoinRegionScanner.nextRaw(HashJoinRegionScanner.java:274)
        at org.apache.phoenix.coprocessor.DelegateRegionScanner.nextRaw(DelegateRegionScanner.java:77)
        at org.apache.phoenix.coprocessor.DelegateRegionScanner.nextRaw(DelegateRegionScanner.java:77)
        at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$RegionScannerHolder.nextRaw(BaseScannerRegionObserver.java:261)
        at org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2555)
        at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:33648)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2170)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:109)
        at org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:133)
        at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:108)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.IllegalStateException: The next hint must come after previous hint (prev=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, next=\x80\x00\x00\x07//LATEST_TIMESTAMP/Maximum/vlen=0/seqid=0, kv=\x80\x00\x00\x06/0:COL1/1487348135903/Put/vlen=2/seqid=4)
        at org.apache.phoenix.filter.SkipScanFilter.setNextCellHint(SkipScanFilter.java:171)
        at org.apache.phoenix.filter.SkipScanFilter.filterKeyValue(SkipScanFilter.java:145)
        at org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.match(ScanQueryMatcher.java:427)
        at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:522)
        at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:147)
        at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:5661)
        at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:5820)
        at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:5598)
        at org.apache.phoenix.coprocessor.BaseScannerRegionObserver$1.nextRaw(BaseScannerRegionObserver.java:459)
        ... 11 more
{code}

That is because {{SkipScanFilter}} can not support {{OrderBy.REV_ROW_KEY_ORDER_BY}} now, if the OrderBy is  {{OrderBy.REV_ROW_KEY_ORDER_BY}}, we can not use {{SkipScanFilter}} after applying  join dynamic filter.

> Incorrect query results when applying inner join and orderby desc
> -----------------------------------------------------------------
>
>                 Key: PHOENIX-3578
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-3578
>             Project: Phoenix
>          Issue Type: Bug
>    Affects Versions: 4.8.0
>         Environment: hbase-1.1.2
>            Reporter: sungmin.cho
>
> Step to reproduce:
> h4. 1. Create two tables
> {noformat}
> CREATE TABLE IF NOT EXISTS master (
>   id integer not null,
>   col1 varchar,
>   constraint pk_master primary key(id)
> );
> CREATE TABLE IF NOT EXISTS detail (
>   id integer not null,
>   seq integer not null,
>   col2 varchar,
>   constraint pk_master primary key(id, seq)
> );
> {noformat}
> h4. 2. Upsert values
> {noformat}
> upsert into master values(1, 'A1');
> upsert into master values(2, 'A2');
> upsert into master values(3, 'A3');
> upsert into detail values(1, 1, 'B1');
> upsert into detail values(1, 2, 'B2');
> upsert into detail values(2, 1, 'B1');
> upsert into detail values(2, 2, 'B2');
> upsert into detail values(3, 1, 'B1');
> upsert into detail values(3, 2, 'B2');
> upsert into detail values(3, 3, 'B3');
> {noformat}
> h4. 3. Execute query
> {noformat}
> select m.id, m.col1, d.seq, d.col2
> from master m, detail d
> where m.id = d.id
>   and d.id between 1 and 2
> order by m.id desc
> {noformat}
> h4. (/) Expected result
> {noformat}
> +-------+---------+--------+---------+
> | M.ID  | M.COL1  | D.SEQ  | D.COL2  |
> +-------+---------+--------+---------+
> | 2     | A2      | 1      | B1      |
> | 2     | A2      | 2      | B2      |
> | 1     | A1      | 1      | B1      |
> | 1     | A1      | 2      | B2      |
> +-------+---------+--------+---------+
> {noformat}
> h4. (!) Incorrect result
> {noformat}
> +-------+---------+--------+---------+
> | M.ID  | M.COL1  | D.SEQ  | D.COL2  |
> +-------+---------+--------+---------+
> | 1     | A1      | 1      | B1      |
> | 1     | A1      | 2      | B2      |
> +-------+---------+--------+---------+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)