[
https://issues.apache.org/jira/browse/PHOENIX-7352?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
chenglei updated PHOENIX-7352:
------------------------------
Description:
This is an improvement for PHOENIX-5148:
Given a table
{code:java}
create table test (
pk1 varchar not null ,
pk2 varchar not null,
pk3 varchar not null,
v1 varchar,
v2 varchar,
CONSTRAINT TEST_PK PRIMARY KEY (
pk1,
pk2,
pk3 ))
{code}
PHOENIX-5148 optimized following sql to compile outer {{OrderBy}} {{order by
v2,pk3}}:
{code:java}
select v1 from (select v1,v2,pk3 from test t where pk1 = '6' order by
t.v2,t.pk3,t.v1 limit 10) a order by v2,pk3
{code}
But if we make a small change to the sql above, just remove the {{t.v1}} in
inner query OrderBy {{order by t.v2,t.pk3,t.v1}}:
{code:java}
select v1 from (select v1,v2,pk3 from test t where pk1 = '6' order by
t.v2,t.pk3 limit 10) a order by v2,pk3
{code}
Obviously, the outer {{OrderBy}} {{order by v2,pk3}} should be compiled out
because it matchs the inner query OrderBy {{order by t.v2,t.pk3}} , but
unfortunately it could not be compiled out now.
was:
This is an improvement for PHOENIX-5148:
Given a table
{code:java}
create table test (
pk1 varchar not null ,
pk2 varchar not null,
pk3 varchar not null,
v1 varchar,
v2 varchar,
CONSTRAINT TEST_PK PRIMARY KEY (
pk1,
pk2,
pk3 ))
{code}
PHOENIX-5148 optimized following sql to compile outer {{OrderBy}} {{order by
v2,pk3}}:
{code:java}
select v1 from (select v1,v2,pk3 from test t where pk1 = '6' order by
t.v2,t.pk3,t.v1 limit 10) a order by v2,pk3
{code}
But if we , the outer {{OrderBy}} "order by v2,pk3" should be compiled out
because it matchs the inner query {{OrderBy}} "order by t.v2,t.pk3,t.v1" , but
unfortunately it is not compiled out.
> Improve OrderPreservingTracker to support extracting partial ordering columns
> for TupleProjectionPlan
> ------------------------------------------------------------------------------------------------------
>
> Key: PHOENIX-7352
> URL: https://issues.apache.org/jira/browse/PHOENIX-7352
> Project: Phoenix
> Issue Type: Improvement
> Components: core
> Affects Versions: 5.2.0
> Reporter: chenglei
> Assignee: chenglei
> Priority: Major
>
> This is an improvement for PHOENIX-5148:
> Given a table
> {code:java}
> create table test (
> pk1 varchar not null ,
> pk2 varchar not null,
> pk3 varchar not null,
> v1 varchar,
> v2 varchar,
> CONSTRAINT TEST_PK PRIMARY KEY (
> pk1,
> pk2,
> pk3 ))
> {code}
> PHOENIX-5148 optimized following sql to compile outer {{OrderBy}} {{order by
> v2,pk3}}:
> {code:java}
> select v1 from (select v1,v2,pk3 from test t where pk1 = '6' order by
> t.v2,t.pk3,t.v1 limit 10) a order by v2,pk3
> {code}
> But if we make a small change to the sql above, just remove the {{t.v1}} in
> inner query OrderBy {{order by t.v2,t.pk3,t.v1}}:
> {code:java}
> select v1 from (select v1,v2,pk3 from test t where pk1 = '6' order by
> t.v2,t.pk3 limit 10) a order by v2,pk3
> {code}
> Obviously, the outer {{OrderBy}} {{order by v2,pk3}} should be compiled out
> because it matchs the inner query OrderBy {{order by t.v2,t.pk3}} , but
> unfortunately it could not be compiled out now.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)