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

chenglei edited comment on PHOENIX-3451 at 11/14/16 3:01 PM:
-------------------------------------------------------------

This bug is caused by the OrderByCompiler,my analysis of this bug is as follows:

take following table which describe by [~jpalmert]  as a example : 
{code:borderStyle=solid} 
     CREATE TABLE IF NOT EXISTS TEST.TEST (
            ORGANIZATION_ID CHAR(15) NOT NULL,
            CONTAINER_ID CHAR(15) NOT NULL,
            ENTITY_ID CHAR(15) NOT NULL,
            SCORE DOUBLE,
            CONSTRAINT TEST_PK PRIMARY KEY (
               ORGANIZATION_ID,
               CONTAINER_ID,
               ENTITY_ID
             )
         )

    CREATE INDEX IF NOT EXISTS TEST_SCORE ON  
TEST.TEST(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC);

    UPSERT INTO test.test VALUES ('org2','container2','entityId6',1.1);
    UPSERT INTO test.test VALUES ('org2','container1','entityId5',1.2);
    UPSERT INTO test.test VALUES ('org2','container2','entityId4',1.3);
    UPSERT INTO test.test VALUES ('org2','container1','entityId3',1.4);
    UPSERT INTO test.test VALUES ('org2','container3','entityId7',1.35);
    UPSERT INTO test.test VALUES ('org2','container3','entityId8',1.45);
{code} 

for the following query sql :
{code:borderStyle=solid} 
    SELECT DISTINCT entity_id, score
    FROM test.test
    WHERE organization_id = 'org2'
    AND container_id IN ( 'container1','container2','container3' )
    ORDER BY score DESC
    LIMIT 2
{code} 

the phoenix would use the following index table  TEST_SCORE to do the query:
{code:borderStyle=solid} 
    CREATE INDEX IF NOT EXISTS TEST_SCORE ON  
TEST.TEST(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC);
{code} 

Using that index is good,the problem is that the OrderByCompiler think the 
OrderBy is OrderBy.FWD_ROW_KEY_ORDER_BY,but because the where condition 
"container_id IN ( 'container1','container2','container3' )", obviously OrderBy 
is not OrderBy.FWD_ROW_KEY_ORDER_BY.

When we look into  OrderByCompiler's compile method, in line 123,  the  "score" 
ColumnParseNode in  "ORDER BY score DESC" accepts a   ExpressionCompiler 
visitor:
{code:borderStyle=solid} 
123                expression = node.getNode().accept(compiler);
124                // Detect mix of aggregate and non aggregates (i.e. ORDER BY 
txns, SUM(txns)
125                if (!expression.isStateless() && !compiler.isAggregate()) {
126                    if (statement.isAggregate() || statement.isDistinct()) {
127                        // Detect ORDER BY not in SELECT DISTINCT: SELECT 
DISTINCT count(*) FROM t ORDER BY x
128                        if (statement.isDistinct()) {
129                            throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.ORDER_BY_NOT_IN_SELECT_DISTINCT)
130                            
.setMessage(expression.toString()).build().buildException();
131                        }
132                        
ExpressionCompiler.throwNonAggExpressionInAggException(expression.toString());
{code} 
In ExpressionCompiler 's visit method,the "score" ColumnParseNode converts to a 
KeyValueColumnExpression in line 408,then in line 409,wrapGroupByExpression 
method is invoked:
{code:borderStyle=solid} 
393  public Expression visit(ColumnParseNode node) throws SQLException {
         ....
408               Expression expression = 
ref.newColumnExpression(node.isTableNameCaseSensitive(), 
node.isCaseSensitive());
409               Expression wrappedExpression = 
wrapGroupByExpression(expression);
{code} 
in wrapGroupByExpression method,because the "score" is in 
groupBy.getExpressions(),which is "[ENTITY_ID, SCORE]",so 
KeyValueColumnExpression  is replaced by RowKeyColumnExpression, because the 
index of "score" in "[ENTITY_ID, SCORE]" is 1,so the return value of 
RowKeyColumnExpression 's position method is 1 :

{code:borderStyle=solid} 
282   private Expression wrapGroupByExpression(Expression expression) {
        .....
286        if (aggregateFunction == null) {
287            int index = groupBy.getExpressions().indexOf(expression);
288            if (index >= 0) {
289                isAggregate = true;
290                RowKeyValueAccessor accessor = new 
RowKeyValueAccessor(groupBy.getKeyExpressions(), index);
291                expression = new RowKeyColumnExpression(expression, 
accessor, groupBy.getKeyExpressions().get(index).getDataType());
292            }
293        }
294        return expression;
295    }
 
{code} 

so when OrderByCompiler's compile method invokes OrderPreservingTracker's track 
method, in line 108,the return Info's pkPosition is 1:

{code:borderStyle=solid} 
106    public void track(Expression node, SortOrder sortOrder, boolean 
isNullsLast) {
107         if (isOrderPreserving) {
108             Info info = node.accept(visitor);
109             if (info == null) {
110                 isOrderPreserving = false;
111            } else {
{code} 

next when OrderByCompiler's compile method invokes OrderPreservingTracker's 
isOrderPreserving method,because pos in line 186 is 1,so the isOrderPreserving 
in line 187 is dependent on hasEqualityConstraints(prevPos+prevSlotSpan, pos) 
method:

{code:borderStyle=solid} 
184   for (int i = 0; i < orderPreservingInfos.size() && isOrderPreserving; 
i++) {
185           Info entry = orderPreservingInfos.get(i);
186            int pos = entry.pkPosition;
187            isOrderPreserving &= entry.orderPreserving != OrderPreserving.NO 
&& prevOrderPreserving == OrderPreserving.YES && (pos == prevPos || pos - 
prevSlotSpan == prevPos  || hasEqualityConstraints(prevPos+prevSlotSpan, pos));
188            prevPos = pos;
189            prevSlotSpan = entry.slotSpan;
190            prevOrderPreserving = entry.orderPreserving;
191        }
{code} 

in following hasEqualityConstraints method, "startPos" parameter is 0,"endPos" 
parameter is 1,and rangs in line 197 is [[org2           ], [container1     , 
container2     , container3     ]], so ranges.hasEqualityConstraint(0) in line 
199 is true,and hasEqualityConstraints method return true, so 
OrderPreservingTracker's  isOrderPreserving  is true,and as a result,OrderBy is 
OrderBy.FWD_ROW_KEY_ORDER_BY.

However, the position of "score" column in  index "TEST_SCORE ON  
TEST.TEST(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC)" is  2, not 
1, 1 is the position of " score" column in GroupBy expressions "SELECT DISTINCT 
entity_id, score" . When hasEqualityConstraints method is called, the "endPos" 
parameter should be 2, not 1. 

{code:borderStyle=solid} 
196  private boolean hasEqualityConstraints(int startPos, int endPos) {
197          ScanRanges ranges = context.getScanRanges();
198           for (int pos = startPos; pos < endPos; pos++) {
199            if (!ranges.hasEqualityConstraint(pos)) {
200                return false;
201            }
202        }
203        return true;
204    }
{code} 

So the root cause is in OrderPreservingTracker's isOrderPreserving method,the 
Info's pkPosition in line 186 should be the position in original RowKey 
Columns, not the position in GroupBy expresssions.



was (Author: comnetwork):
This bug is caused by the OrderByCompiler,my analysis of this bug is as follows:

take following table which describe by [~jpalmert]  as a example : 
{code:borderStyle=solid} 
     CREATE TABLE IF NOT EXISTS TEST.TEST (
            ORGANIZATION_ID CHAR(15) NOT NULL,
            CONTAINER_ID CHAR(15) NOT NULL,
            ENTITY_ID CHAR(15) NOT NULL,
            SCORE DOUBLE,
            CONSTRAINT TEST_PK PRIMARY KEY (
               ORGANIZATION_ID,
               CONTAINER_ID,
               ENTITY_ID
             )
         )

    CREATE INDEX IF NOT EXISTS TEST_SCORE ON  
TEST.TEST(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC);

    UPSERT INTO test.test VALUES ('org2','container2','entityId6',1.1);
    UPSERT INTO test.test VALUES ('org2','container1','entityId5',1.2);
    UPSERT INTO test.test VALUES ('org2','container2','entityId4',1.3);
    UPSERT INTO test.test VALUES ('org2','container1','entityId3',1.4);
    UPSERT INTO test.test VALUES ('org2','container3','entityId7',1.35);
    UPSERT INTO test.test VALUES ('org2','container3','entityId8',1.45);
{code} 

for the following select sql :
{code:borderStyle=solid} 
    SELECT DISTINCT entity_id, score
    FROM test.test
    WHERE organization_id = 'org2'
    AND container_id IN ( 'container1','container2','container3' )
    ORDER BY score DESC
    LIMIT 2
{code} 

the phoenix would use the following index table  TEST_SCORE to do the query:
{code:borderStyle=solid} 
    CREATE INDEX IF NOT EXISTS TEST_SCORE ON  
TEST.TEST(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC);
{code} 

Using that index is good,the problem is that the OrderByCompiler think the 
OrderBy is OrderBy.FWD_ROW_KEY_ORDER_BY,but because the where condition 
"container_id IN ( 'container1','container2','container3' )", obviously OrderBy 
is not OrderBy.FWD_ROW_KEY_ORDER_BY.

When we look into  OrderByCompiler's compile method, in line 123,  the  "score" 
ColumnParseNode in  "ORDER BY score DESC" accepts a   ExpressionCompiler 
visitor:
{code:borderStyle=solid} 
123                expression = node.getNode().accept(compiler);
124                // Detect mix of aggregate and non aggregates (i.e. ORDER BY 
txns, SUM(txns)
125                if (!expression.isStateless() && !compiler.isAggregate()) {
126                    if (statement.isAggregate() || statement.isDistinct()) {
127                        // Detect ORDER BY not in SELECT DISTINCT: SELECT 
DISTINCT count(*) FROM t ORDER BY x
128                        if (statement.isDistinct()) {
129                            throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.ORDER_BY_NOT_IN_SELECT_DISTINCT)
130                            
.setMessage(expression.toString()).build().buildException();
131                        }
132                        
ExpressionCompiler.throwNonAggExpressionInAggException(expression.toString());
{code} 
In ExpressionCompiler 's visit method,the "score" ColumnParseNode converts to a 
KeyValueColumnExpression in line 408,then in line 409,wrapGroupByExpression 
method is invoked:
{code:borderStyle=solid} 
393  public Expression visit(ColumnParseNode node) throws SQLException {
         ....
408               Expression expression = 
ref.newColumnExpression(node.isTableNameCaseSensitive(), 
node.isCaseSensitive());
409               Expression wrappedExpression = 
wrapGroupByExpression(expression);
{code} 
in wrapGroupByExpression method,because the "score" is in 
groupBy.getExpressions(),which is "[ENTITY_ID, SCORE]",so 
KeyValueColumnExpression  is replaced by RowKeyColumnExpression, because the 
index of "score" in "[ENTITY_ID, SCORE]" is 1,so the return value of 
RowKeyColumnExpression 's position method is 1 :

{code:borderStyle=solid} 
282   private Expression wrapGroupByExpression(Expression expression) {
        .....
286        if (aggregateFunction == null) {
287            int index = groupBy.getExpressions().indexOf(expression);
288            if (index >= 0) {
289                isAggregate = true;
290                RowKeyValueAccessor accessor = new 
RowKeyValueAccessor(groupBy.getKeyExpressions(), index);
291                expression = new RowKeyColumnExpression(expression, 
accessor, groupBy.getKeyExpressions().get(index).getDataType());
292            }
293        }
294        return expression;
295    }
 
{code} 

so when OrderByCompiler's compile method invokes OrderPreservingTracker's track 
method, in line 108,the return Info's pkPosition is 1:

{code:borderStyle=solid} 
106    public void track(Expression node, SortOrder sortOrder, boolean 
isNullsLast) {
107         if (isOrderPreserving) {
108             Info info = node.accept(visitor);
109             if (info == null) {
110                 isOrderPreserving = false;
111            } else {
{code} 

next when OrderByCompiler's compile method invokes OrderPreservingTracker's 
isOrderPreserving method,because pos in line 186 is 1,so the isOrderPreserving 
in line 187 is dependent on hasEqualityConstraints(prevPos+prevSlotSpan, pos) 
method:

{code:borderStyle=solid} 
184   for (int i = 0; i < orderPreservingInfos.size() && isOrderPreserving; 
i++) {
185           Info entry = orderPreservingInfos.get(i);
186            int pos = entry.pkPosition;
187            isOrderPreserving &= entry.orderPreserving != OrderPreserving.NO 
&& prevOrderPreserving == OrderPreserving.YES && (pos == prevPos || pos - 
prevSlotSpan == prevPos  || hasEqualityConstraints(prevPos+prevSlotSpan, pos));
188            prevPos = pos;
189            prevSlotSpan = entry.slotSpan;
190            prevOrderPreserving = entry.orderPreserving;
191        }
{code} 

in following hasEqualityConstraints method, "startPos" parameter is 0,"endPos" 
parameter is 1,and rangs in line 197 is [[org2           ], [container1     , 
container2     , container3     ]], so ranges.hasEqualityConstraint(0) in line 
199 is true,and hasEqualityConstraints method return true, so 
OrderPreservingTracker's  isOrderPreserving  is true,and as a result,OrderBy is 
OrderBy.FWD_ROW_KEY_ORDER_BY.

However, the position of "score" column in  index "TEST_SCORE ON  
TEST.TEST(ORGANIZATION_ID,CONTAINER_ID, SCORE DESC, ENTITY_ID DESC)" is  2, not 
1, 1 is the position of " score" column in GroupBy expressions "SELECT DISTINCT 
entity_id, score" . When hasEqualityConstraints method is called, the "endPos" 
parameter should be 2, not 1. 

{code:borderStyle=solid} 
196  private boolean hasEqualityConstraints(int startPos, int endPos) {
197          ScanRanges ranges = context.getScanRanges();
198           for (int pos = startPos; pos < endPos; pos++) {
199            if (!ranges.hasEqualityConstraint(pos)) {
200                return false;
201            }
202        }
203        return true;
204    }
{code} 

So the root cause is in OrderPreservingTracker's isOrderPreserving method,the 
Info's pkPosition in line 186 should be the position in original RowKey 
Columns, not the position in GroupBy expresssions.


> Secondary index and query using distinct: LIMIT doesn't return the first rows
> -----------------------------------------------------------------------------
>
>                 Key: PHOENIX-3451
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-3451
>             Project: Phoenix
>          Issue Type: Bug
>    Affects Versions: 4.8.0
>            Reporter: Joel Palmert
>            Assignee: chenglei
>
> This may be related to PHOENIX-3452 but the behavior is different so filing 
> it separately.
> Steps to repro:
> CREATE TABLE IF NOT EXISTS TEST.TEST (
>     ORGANIZATION_ID CHAR(15) NOT NULL,
>     CONTAINER_ID CHAR(15) NOT NULL,
>     ENTITY_ID CHAR(15) NOT NULL,
>     SCORE DOUBLE,
>     CONSTRAINT TEST_PK PRIMARY KEY (
>         ORGANIZATION_ID,
>         CONTAINER_ID,
>         ENTITY_ID
>     )
> ) VERSIONS=1, MULTI_TENANT=TRUE, REPLICATION_SCOPE=1, TTL=31536000;
> CREATE INDEX IF NOT EXISTS TEST_SCORE ON TEST.TEST (CONTAINER_ID, SCORE DESC, 
> ENTITY_ID DESC);
> UPSERT INTO test.test VALUES ('org2','container2','entityId6',1.1);
> UPSERT INTO test.test VALUES ('org2','container1','entityId5',1.2);
> UPSERT INTO test.test VALUES ('org2','container2','entityId4',1.3);
> UPSERT INTO test.test VALUES ('org2','container1','entityId3',1.4);
> UPSERT INTO test.test VALUES ('org2','container3','entityId7',1.35);
> UPSERT INTO test.test VALUES ('org2','container3','entityId8',1.45);
> EXPLAIN
> SELECT DISTINCT entity_id, score
> FROM test.test
> WHERE organization_id = 'org2'
> AND container_id IN ( 'container1','container2','container3' )
> ORDER BY score DESC
> LIMIT 2
> OUTPUT
> entityId5    1.2
> entityId3    1.4
> The expected out out would be
> entityId8    1.45
> entityId3    1.4
> You will get the expected output if you remove the secondary index from the 
> table or remove distinct from the query.
> As described in PHOENIX-3452 if you run the query without the LIMIT the 
> ordering is not correct. However, the 2first results in that ordering is 
> still not the onces returned by the limit clause, which makes me think there 
> are multiple issues here and why I filed both separately. The rows being 
> returned are the ones assigned to container1. It looks like Phoenix is first 
> getting the rows from the first container and when it finds that to be enough 
> it stops the scan. What it should be doing is getting 2 results for each 
> container and then merge then and then limit again.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to