[jira] [Commented] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread James Taylor (JIRA)

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

James Taylor commented on PHOENIX-4283:
---

The coercion is done on types that don't have a representation for null, like 
BIGINT, since you might not find any values. In this particular case, since A 
and C are not null, there shouldn't be any coercion (it really should be able 
to just optimize out the outer group by), but I don't think that's where the 
problem is (though that would impact perf). It should be fine to convert from 
BIGINT->DECIMAL->BIGINT without losing any precision.

For a nested group by (i.e. where there's an inner and outer group by), the 
outer group by is evaluated on the client side. There must be an issue there, 
as without the nesting it works fine. Maybe we get the type information wrong? 
Or maybe we try to optimize out the group by, but mess something up in the 
process.

> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread Ethan Wang (JIRA)

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

Ethan Wang edited comment on PHOENIX-4283 at 10/14/17 1:27 AM:
---

So in the GroupbyCompiler, when nested groupby is evaluated, in this case it 
will try to coarse all the leading ProjectedColumnExpressions (except the last 
one) inside the expressions to the required types. e.g., for (A,C) it will be 
(A); for (A,B,C,D,E) it will be (A,B,C,D) force to convert.

When doing so, in IndexUtil.getIndexColumnDataType(), BigInt (PLong) is defined 
to be coarsable with PDecimal. (in PLong.isComparableTo() ). As a result, all 
the leading BIGINT ProjectedColumnExpressions is now casted to decimal. The 
Decimal will be converted back to the appropriate type (Integer or Long). 

If I understand right the reason behind this is that it needs something to give 
coprocessor sort by in the case groupbys is not along the PK axis. However, 
when a region have no such groupbys, it need null to hold on. And decimal is 
the "appropriate" type that we can construct a such a null. Is this correct?  



was (Author: aertoria):
So in the GroupbyCompiler, when nested groupby is evaluated, it will try to 
coarse all the leading   ProjectedColumnExpressions (except the last one) 
inside the expressions to the required types. e.g., for (A,C) it will be (A); 
for (A,B,C,D,E) it will be (A,B,C,D) force to convert.

When doing so, in IndexUtil.getIndexColumnDataType(), BigInt (PLong) is defined 
to be coarsable with PDecimal. (in PLong.isComparableTo() ). As a result, all 
the leading BIGINT ProjectedColumnExpressions is now casted to decimal. The 
Decimal will be converted back to the appropriate type (Integer or Long). 

If I understand right the reason behind this is that it needs something to give 
coprocessor sort by in the case groupbys is not along the PK axis. However, 
when a region have no such groupbys, it need null to hold on. And decimal is 
the "appropriate" type that we can construct a such a null. Is this correct?  


> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread Ethan Wang (JIRA)

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

Ethan Wang edited comment on PHOENIX-4283 at 10/14/17 1:08 AM:
---

So in the GroupbyCompiler, when nested groupby is evaluated, it will try to 
coarse all the leading   ProjectedColumnExpressions (except the last one) 
inside the expressions to the required types. e.g., for (A,C) it will be (A); 
for (A,B,C,D,E) it will be (A,B,C,D) force to convert.

When doing so, in IndexUtil.getIndexColumnDataType(), BigInt (PLong) is defined 
to be coarsable with PDecimal. (in PLong.isComparableTo() ). As a result, all 
the leading BIGINT ProjectedColumnExpressions is now casted to decimal. The 
Decimal will be converted back to the appropriate type (Integer or Long). 

If I understand right the reason behind this is that it needs something to give 
coprocessor sort by in the case groupbys is not along the PK axis. However, 
when a region have no such groupbys, it need null to hold on. And decimal is 
the "appropriate" type that we can construct a such a null. Is this correct?  



was (Author: aertoria):
So in the GroupbyCompiler, when nested groupby is evaluated, it will try to 
coarse all the leading   ProjectedColumnExpressions (except the last one) 
inside the expressions to the required types. e.g., for (A,C) it will be (A); 
for (A,B,C,D,E) it will be (A,B,C,D) force to convert.

When doing so, in IndexUtil.getIndexColumnDataType(), BigInt (PLong) is defined 
to be coarsable with PDecimal. (in PLong.isComparableTo() ). As a result, all 
the leading BIGINT ProjectedColumnExpressions is now casted to decimal. The 
Decimal will be converted back to the appropriate type (Integer or Long). 

If I understand right the reason behind this is that it needs something to give 
coprocessor sort by in the case groupbys is not along the PK axis. However, 
when a region have no such groupbys, it need null to hold on. And decimal is 
the "appropriate" type that we can construct a such a null. 


> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread Ethan Wang (JIRA)

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

Ethan Wang edited comment on PHOENIX-4283 at 10/14/17 1:04 AM:
---

So in the GroupbyCompiler, when nested groupby is evaluated, it will try to 
coarse all the leading   ProjectedColumnExpressions (except the last one) 
inside the expressions to the required types. e.g., for (A,C) it will be (A); 
for (A,B,C,D,E) it will be (A,B,C,D) force to convert.

When doing so, in IndexUtil.getIndexColumnDataType(), BigInt (PLong) is defined 
to be coarsable with PDecimal. (in PLong.isComparableTo() ). As a result, all 
the leading BIGINT ProjectedColumnExpressions is now casted to decimal. The 
Decimal will be converted back to the appropriate type (Integer or Long). 

If I understand right the reason behind this is that it needs something to give 
coprocessor sort by in the case groupbys is not along the PK axis. However, 
when a region have no such groupbys, it need null to hold on. And decimal is 
the "appropriate" type that we can construct a such a null. 



was (Author: aertoria):
So in the GroupbyCompiler, when nested groupby is evaluated, it will try to 
coarse all the leading   ProjectedColumnExpressions (except the last one) 
inside the expressions to the required types. e.g., for (A,C) it will be (A); 
for (A,B,C,D,E) it will be (A,B,C,D) force to convert.

When doing so, in IndexUtil.getIndexColumnDataType(), BigInt (PLong) is defined 
to be coarsable with PDecimal. (in PLong.isComparableTo() ). As a result, all 
the leading BIGINT ProjectedColumnExpressions is now casted to decimal.



> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread Ethan Wang (JIRA)

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

Ethan Wang commented on PHOENIX-4283:
-

So in the GroupbyCompiler, when nested groupby is evaluated, it will try to 
coarse all the leading   ProjectedColumnExpressions (except the last one) 
inside the expressions to the required types. e.g., for (A,C) it will be (A); 
for (A,B,C,D,E) it will be (A,B,C,D) force to convert.

When doing so, in IndexUtil.getIndexColumnDataType(), BigInt (PLong) is defined 
to be coarsable with PDecimal. (in PLong.isComparableTo() ). As a result, all 
the leading BIGINT ProjectedColumnExpressions is now casted to decimal.



> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [DISCUSS] Road to HBase 2.0.0

2017-10-13 Thread Sergey Soldatov
There is a list of problems that we need to fix to get it working with
HBase 2.0:
1. renamed methods (such as add => addColumn). That's the easiest to fix
2. removed interfaces such as HTableInterface. We are supposed to use Table
now. That may lead to some small difference between branches. Because
that's funny, but there are some HBase 1.x methods are using deprecated
API, but have no duplicated methods that appears only in 2.0. The same
thing with KeyValue => Cell if I remember correctly.
3. Our wrappers for HTable/etc. Since the set of methods sometimes very
different for 1.x and 2.0, we will need to maintain different versions.
Luckily that's one time work and maintaining would not be hard because we
rarely change them.
4. Some renaming would be required. I can recall KeyValueUtil, possible
some else to avoid ambiguity between our and HBase classes.
5. signature some of methods has been changed. Like batch() now doesn't
return the result, but requires it as an argument
6. Many methods switched from using byte[] to TableName. Don't remember,
but possible that works fine with 1.x, so it can be part of (1)
refactoring.
7. new Coprocessors. I have no idea how many changes we will have, because
my previous attempts to get it working with HBase 2.0 were far before new
implementation.

I support the roadmap provided by James. Most of the changes I listed
(300+K patch) can be done as the first bullet in his list.

Thanks,
Sergey

On Fri, Oct 13, 2017 at 2:40 PM, James Taylor 
wrote:

> One idea of where to put the code:
> - switch to using non deprecated HBase methods in master
> - same for Tephra
> - create a 5.0-HBase-2.0 branch to put code specific to getting Phoenix to
> work against HBase 2.0
> - take a look at the changes and see if a shim layer makes sense (I'm only
> -1 on an HBase 0.98 shim layer because the life span of that branch is very
> limited)
> - eventually make 5.0-HBase-2.0 the master branch
>
> Thoughts?
>
> On Fri, Oct 13, 2017 at 8:31 AM, Josh Elser  wrote:
>
> > Thanks, Anoop!
> >
> > I know Sergey, Ankit, and Rajeshbabu have been looking at this already.
> >
> > While tracking it is good, I think we still need to come up with a plan
> > for where we're going to put that new code in Phoenix :)
> >
> >
> > On 10/13/17 6:58 AM, Anoop John wrote:
> >
> >> Thanks for bringing this up.  I was abt to ask this.  Ya the CP
> >> framework itself and the CP exposed interfaces (Like
> >> RegionServerServices, Region etc) are undergoing a big change for for
> >> HBase 2.0..  I did  a look at some of the usages of the Phoenix
> >> exposed interfaces/classes.  There are some items for fix.   Was
> >> thinking to raise an umbrella issue once we have a plan for the
> >> version based on HBase 2.0
> >>
> >> -Anoop-
> >>
> >> On Thu, Oct 12, 2017 at 3:30 AM, Josh Elser  wrote:
> >>
> >>> Since 4.12.0 is out and we have the concurrent discussions about the
> >>> 0.98,
> >>> 1.1, and 1.2 HBase branches, do folks have a vision of how we get to
> >>> HBase
> >>> 2.0.0?
> >>>
> >>> The lack of chatter is pretty obvious that the Calcite work (the
> previous
> >>> impetus for Phoenix 5) has slowed. Once we get to an HBase
> 2.0.0-alpha4,
> >>> coprocessor API should stabilize and give us a point against which we
> can
> >>> start Phoenix work.
> >>>
> >>> Should a release of Phoenix that supports HBase 2.0 be worthy of the
> >>> Phoenix
> >>> 5.0 label, or should we stick to the 4.x numbering? Given the breaking
> >>> changes going into HBase 2.0 and James' previous -1 to shim-layers for
> >>> 0.98,
> >>> do we see the same for an HBase 2.0 branch or is HBase 1.x/2.x a
> >>> different
> >>> beast? I can see pros/cons for both sides.
> >>>
> >>> - Josh
> >>>
> >>
>


[jira] [Commented] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread James Taylor (JIRA)

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

James Taylor commented on PHOENIX-4283:
---

Ok, good to know. Hopefully it's an easy fix.

> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [DISCUSS] Road to HBase 2.0.0

2017-10-13 Thread James Taylor
One idea of where to put the code:
- switch to using non deprecated HBase methods in master
- same for Tephra
- create a 5.0-HBase-2.0 branch to put code specific to getting Phoenix to
work against HBase 2.0
- take a look at the changes and see if a shim layer makes sense (I'm only
-1 on an HBase 0.98 shim layer because the life span of that branch is very
limited)
- eventually make 5.0-HBase-2.0 the master branch

Thoughts?

On Fri, Oct 13, 2017 at 8:31 AM, Josh Elser  wrote:

> Thanks, Anoop!
>
> I know Sergey, Ankit, and Rajeshbabu have been looking at this already.
>
> While tracking it is good, I think we still need to come up with a plan
> for where we're going to put that new code in Phoenix :)
>
>
> On 10/13/17 6:58 AM, Anoop John wrote:
>
>> Thanks for bringing this up.  I was abt to ask this.  Ya the CP
>> framework itself and the CP exposed interfaces (Like
>> RegionServerServices, Region etc) are undergoing a big change for for
>> HBase 2.0..  I did  a look at some of the usages of the Phoenix
>> exposed interfaces/classes.  There are some items for fix.   Was
>> thinking to raise an umbrella issue once we have a plan for the
>> version based on HBase 2.0
>>
>> -Anoop-
>>
>> On Thu, Oct 12, 2017 at 3:30 AM, Josh Elser  wrote:
>>
>>> Since 4.12.0 is out and we have the concurrent discussions about the
>>> 0.98,
>>> 1.1, and 1.2 HBase branches, do folks have a vision of how we get to
>>> HBase
>>> 2.0.0?
>>>
>>> The lack of chatter is pretty obvious that the Calcite work (the previous
>>> impetus for Phoenix 5) has slowed. Once we get to an HBase 2.0.0-alpha4,
>>> coprocessor API should stabilize and give us a point against which we can
>>> start Phoenix work.
>>>
>>> Should a release of Phoenix that supports HBase 2.0 be worthy of the
>>> Phoenix
>>> 5.0 label, or should we stick to the 4.x numbering? Given the breaking
>>> changes going into HBase 2.0 and James' previous -1 to shim-layers for
>>> 0.98,
>>> do we see the same for an HBase 2.0 branch or is HBase 1.x/2.x a
>>> different
>>> beast? I can see pros/cons for both sides.
>>>
>>> - Josh
>>>
>>


[jira] [Commented] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread Steven Sadowski (JIRA)

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

Steven Sadowski commented on PHOENIX-4283:
--

Yeah, we're using that work-around now. Should be fine for the near-term future.

> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PHOENIX-4287) Incorrect aggregate query results when stats are disable for parallelization

2017-10-13 Thread James Taylor (JIRA)

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

James Taylor commented on PHOENIX-4287:
---

count( * ) uses the index because it's smaller in size than the data table (and 
has the same number of rows).

> Incorrect aggregate query results when stats are disable for parallelization
> 
>
> Key: PHOENIX-4287
> URL: https://issues.apache.org/jira/browse/PHOENIX-4287
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.12.0
> Environment: HBase 1.3.1
>Reporter: Mujtaba Chohan
>Assignee: Samarth Jain
>
> With {{phoenix.use.stats.parallelization}} set to {{false}}, aggregate query 
> returns incorrect results when stats are available.
> With local index and stats disabled for parallelization:
> {noformat}
> explain select count(*) from TABLE_T;
> +---+-++---+
> | PLAN
>   | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO |
> +---+-++---+
> | CLIENT 0-CHUNK 332170 ROWS 625043899 BYTES PARALLEL 0-WAY RANGE SCAN OVER 
> TABLE_T [1]  | 625043899   | 332170 | 150792825 |
> | SERVER FILTER BY FIRST KEY ONLY 
>   | 625043899   | 332170 | 150792825 |
> | SERVER AGGREGATE INTO SINGLE ROW
>   | 625043899   | 332170 | 150792825 |
> +---+-++---+
> select count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 0 |
> +---+
> {noformat}
> Using data table
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-+++
> |   PLAN  
>  | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO_TS   |
> +--+-+++
> | CLIENT 2-CHUNK 332151 ROWS 438492470 BYTES PARALLEL 1-WAY FULL SCAN OVER 
> TABLE_T  | 438492470   | 332151 | 1507928257617  |
> | SERVER FILTER BY FIRST KEY ONLY 
>  | 438492470   | 332151 | 1507928257617  |
> | SERVER AGGREGATE INTO SINGLE ROW
>  | 438492470   | 332151 | 1507928257617  |
> +--+-+++
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 14|
> +---+
> {noformat}
> Without stats available, results are correct:
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-++--+
> | PLAN | 
> EST_BYTES_READ  | EST_ROWS_READ  | EST_INFO_TS  |
> +--+-++--+
> | CLIENT 2-CHUNK PARALLEL 1-WAY FULL SCAN OVER TABLE_T  | null| 
> null   | null |
> | SERVER FILTER BY FIRST KEY ONLY  | null 
>| null   | null |
> | SERVER AGGREGATE INTO SINGLE ROW | null 
>| null   | null |
> +--+-++--+
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 27|
> +---+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread Ethan Wang (JIRA)

 [ 
https://issues.apache.org/jira/browse/PHOENIX-4283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Wang reassigned PHOENIX-4283:
---

Assignee: Ethan Wang

> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
>Assignee: Ethan Wang
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (PHOENIX-4287) Incorrect aggregate query results when stats are disable for parallelization

2017-10-13 Thread Ethan Wang (JIRA)

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

Ethan Wang edited comment on PHOENIX-4287 at 10/13/17 9:28 PM:
---

Count( *) should always count on pk correct? if that's the case why `select 
count( *) from TABLE_T` tried to use index first?


was (Author: aertoria):
Count(*) should always count on pk correct? if that's the case why `select 
count(*) from TABLE_T` tried to use index first?

> Incorrect aggregate query results when stats are disable for parallelization
> 
>
> Key: PHOENIX-4287
> URL: https://issues.apache.org/jira/browse/PHOENIX-4287
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.12.0
> Environment: HBase 1.3.1
>Reporter: Mujtaba Chohan
>Assignee: Samarth Jain
>
> With {{phoenix.use.stats.parallelization}} set to {{false}}, aggregate query 
> returns incorrect results when stats are available.
> With local index and stats disabled for parallelization:
> {noformat}
> explain select count(*) from TABLE_T;
> +---+-++---+
> | PLAN
>   | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO |
> +---+-++---+
> | CLIENT 0-CHUNK 332170 ROWS 625043899 BYTES PARALLEL 0-WAY RANGE SCAN OVER 
> TABLE_T [1]  | 625043899   | 332170 | 150792825 |
> | SERVER FILTER BY FIRST KEY ONLY 
>   | 625043899   | 332170 | 150792825 |
> | SERVER AGGREGATE INTO SINGLE ROW
>   | 625043899   | 332170 | 150792825 |
> +---+-++---+
> select count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 0 |
> +---+
> {noformat}
> Using data table
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-+++
> |   PLAN  
>  | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO_TS   |
> +--+-+++
> | CLIENT 2-CHUNK 332151 ROWS 438492470 BYTES PARALLEL 1-WAY FULL SCAN OVER 
> TABLE_T  | 438492470   | 332151 | 1507928257617  |
> | SERVER FILTER BY FIRST KEY ONLY 
>  | 438492470   | 332151 | 1507928257617  |
> | SERVER AGGREGATE INTO SINGLE ROW
>  | 438492470   | 332151 | 1507928257617  |
> +--+-+++
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 14|
> +---+
> {noformat}
> Without stats available, results are correct:
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-++--+
> | PLAN | 
> EST_BYTES_READ  | EST_ROWS_READ  | EST_INFO_TS  |
> +--+-++--+
> | CLIENT 2-CHUNK PARALLEL 1-WAY FULL SCAN OVER TABLE_T  | null| 
> null   | null |
> | SERVER FILTER BY FIRST KEY ONLY  | null 
>| null   | null |
> | SERVER AGGREGATE INTO SINGLE ROW | null 
>| null   | null |
> +--+-++--+
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 27|
> +---+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PHOENIX-4287) Incorrect aggregate query results when stats are disable for parallelization

2017-10-13 Thread Ethan Wang (JIRA)

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

Ethan Wang commented on PHOENIX-4287:
-

Count(*) should always count on pk correct? if that's the case why `select 
count(*) from TABLE_T` tried to use index first?

> Incorrect aggregate query results when stats are disable for parallelization
> 
>
> Key: PHOENIX-4287
> URL: https://issues.apache.org/jira/browse/PHOENIX-4287
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.12.0
> Environment: HBase 1.3.1
>Reporter: Mujtaba Chohan
>Assignee: Samarth Jain
>
> With {{phoenix.use.stats.parallelization}} set to {{false}}, aggregate query 
> returns incorrect results when stats are available.
> With local index and stats disabled for parallelization:
> {noformat}
> explain select count(*) from TABLE_T;
> +---+-++---+
> | PLAN
>   | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO |
> +---+-++---+
> | CLIENT 0-CHUNK 332170 ROWS 625043899 BYTES PARALLEL 0-WAY RANGE SCAN OVER 
> TABLE_T [1]  | 625043899   | 332170 | 150792825 |
> | SERVER FILTER BY FIRST KEY ONLY 
>   | 625043899   | 332170 | 150792825 |
> | SERVER AGGREGATE INTO SINGLE ROW
>   | 625043899   | 332170 | 150792825 |
> +---+-++---+
> select count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 0 |
> +---+
> {noformat}
> Using data table
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-+++
> |   PLAN  
>  | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO_TS   |
> +--+-+++
> | CLIENT 2-CHUNK 332151 ROWS 438492470 BYTES PARALLEL 1-WAY FULL SCAN OVER 
> TABLE_T  | 438492470   | 332151 | 1507928257617  |
> | SERVER FILTER BY FIRST KEY ONLY 
>  | 438492470   | 332151 | 1507928257617  |
> | SERVER AGGREGATE INTO SINGLE ROW
>  | 438492470   | 332151 | 1507928257617  |
> +--+-+++
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 14|
> +---+
> {noformat}
> Without stats available, results are correct:
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-++--+
> | PLAN | 
> EST_BYTES_READ  | EST_ROWS_READ  | EST_INFO_TS  |
> +--+-++--+
> | CLIENT 2-CHUNK PARALLEL 1-WAY FULL SCAN OVER TABLE_T  | null| 
> null   | null |
> | SERVER FILTER BY FIRST KEY ONLY  | null 
>| null   | null |
> | SERVER AGGREGATE INTO SINGLE ROW | null 
>| null   | null |
> +--+-++--+
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 27|
> +---+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread James Taylor (JIRA)

 [ 
https://issues.apache.org/jira/browse/PHOENIX-4283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

James Taylor updated PHOENIX-4283:
--
Fix Version/s: 4.12.1

> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PHOENIX-4283) Group By statement truncating BIGINTs

2017-10-13 Thread James Taylor (JIRA)

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

James Taylor commented on PHOENIX-4283:
---

[~aertoria] - would you have some cycles to take a look? I've confirmed it's an 
issue in 4.12 using the regular thick driver too. It works fine without the 
nested query:
{code}
0: jdbc:phoenix:localhost:2181:/hbase> UPSERT INTO test_table(a,c) 
VALUES(444, 555);
1 row affected (0.033 seconds)
0: jdbc:phoenix:localhost:2181:/hbase> SELECT a FROM (SELECT a, c FROM 
test_table GROUP BY a, c) GROUP BY a, c;
+--+
|  A   |
+--+
| 400  |
+--+
1 row selected (0.039 seconds)
0: jdbc:phoenix:localhost:2181:/hbase> SELECT a, c FROM test_table GROUP BY a, 
c;
+--+--+
|  A   |  C   |
+--+--+
| 444  | 555  |
+--+--+
1 row selected (0.04 seconds)
{code}

Is that a possible work around for you, [~ssadowski] while we investigate this?

> Group By statement truncating BIGINTs
> -
>
> Key: PHOENIX-4283
> URL: https://issues.apache.org/jira/browse/PHOENIX-4283
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.11.0
>Reporter: Steven Sadowski
> Fix For: 4.12.1
>
>
> *Versions:*
> Phoenix 4.11.0
> HBase: 1.3.1
> (Amazon EMR: 5.8.0)
> *Steps to reproduce:*
> 1. From the `sqlline-thin.py` client setup the following table:
> {code:sql}
> CREATE TABLE test_table (
> a BIGINT NOT NULL, 
> c BIGINT NOT NULL
> CONSTRAINT PK PRIMARY KEY (a, c)
> );
> UPSERT INTO test_table(a,c) VALUES(444, 555);
> SELECT a FROM (SELECT a, c FROM test_table GROUP BY a, c) GROUP BY a, c;
> {code}
> *Expected Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 444  |
> +--+
> {code}
> *Actual Result:*
> {code:sql}
> +--+
> |  A   |
> +--+
> | 400  |
> +--+
> {code}
> *Comments:*
> Having the two Group By statements together seems to truncate the last 6 or 
> so digits of the final result. Removing the outer (or either) group by will 
> produce the correct result.
> Please fix the Group by statement to not truncate the outer result's value.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PHOENIX-4287) Incorrect aggregate query results when stats are disable for parallelization

2017-10-13 Thread Mujtaba Chohan (JIRA)

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

Mujtaba Chohan commented on PHOENIX-4287:
-

Results are incorrect for filter on leading/non-leading PK as well. Ex {{select 
/**+NO_INDEX**/ * from T where leadingKey='x'}}

> Incorrect aggregate query results when stats are disable for parallelization
> 
>
> Key: PHOENIX-4287
> URL: https://issues.apache.org/jira/browse/PHOENIX-4287
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.12.0
> Environment: HBase 1.3.1
>Reporter: Mujtaba Chohan
>Assignee: Samarth Jain
>
> With {{phoenix.use.stats.parallelization}} set to {{false}}, aggregate query 
> returns incorrect results when stats are available.
> With local index and stats disabled for parallelization:
> {noformat}
> explain select count(*) from TABLE_T;
> +---+-++---+
> | PLAN
>   | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO |
> +---+-++---+
> | CLIENT 0-CHUNK 332170 ROWS 625043899 BYTES PARALLEL 0-WAY RANGE SCAN OVER 
> TABLE_T [1]  | 625043899   | 332170 | 150792825 |
> | SERVER FILTER BY FIRST KEY ONLY 
>   | 625043899   | 332170 | 150792825 |
> | SERVER AGGREGATE INTO SINGLE ROW
>   | 625043899   | 332170 | 150792825 |
> +---+-++---+
> select count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 0 |
> +---+
> {noformat}
> Using data table
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-+++
> |   PLAN  
>  | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO_TS   |
> +--+-+++
> | CLIENT 2-CHUNK 332151 ROWS 438492470 BYTES PARALLEL 1-WAY FULL SCAN OVER 
> TABLE_T  | 438492470   | 332151 | 1507928257617  |
> | SERVER FILTER BY FIRST KEY ONLY 
>  | 438492470   | 332151 | 1507928257617  |
> | SERVER AGGREGATE INTO SINGLE ROW
>  | 438492470   | 332151 | 1507928257617  |
> +--+-+++
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 14|
> +---+
> {noformat}
> Without stats available, results are correct:
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-++--+
> | PLAN | 
> EST_BYTES_READ  | EST_ROWS_READ  | EST_INFO_TS  |
> +--+-++--+
> | CLIENT 2-CHUNK PARALLEL 1-WAY FULL SCAN OVER TABLE_T  | null| 
> null   | null |
> | SERVER FILTER BY FIRST KEY ONLY  | null 
>| null   | null |
> | SERVER AGGREGATE INTO SINGLE ROW | null 
>| null   | null |
> +--+-++--+
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 27|
> +---+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (PHOENIX-4287) Incorrect aggregate query results when stats are disable for parallelization

2017-10-13 Thread Mujtaba Chohan (JIRA)

 [ 
https://issues.apache.org/jira/browse/PHOENIX-4287?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Mujtaba Chohan reassigned PHOENIX-4287:
---

Assignee: Samarth Jain

> Incorrect aggregate query results when stats are disable for parallelization
> 
>
> Key: PHOENIX-4287
> URL: https://issues.apache.org/jira/browse/PHOENIX-4287
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.12.0
> Environment: HBase 1.3.1
>Reporter: Mujtaba Chohan
>Assignee: Samarth Jain
>
> With {{phoenix.use.stats.parallelization}} set to {{false}}, aggregate query 
> returns incorrect results when stats are available.
> With local index and stats disabled for parallelization:
> {noformat}
> explain select count(*) from TABLE_T;
> +---+-++---+
> | PLAN
>   | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO |
> +---+-++---+
> | CLIENT 0-CHUNK 332170 ROWS 625043899 BYTES PARALLEL 0-WAY RANGE SCAN OVER 
> TABLE_T [1]  | 625043899   | 332170 | 150792825 |
> | SERVER FILTER BY FIRST KEY ONLY 
>   | 625043899   | 332170 | 150792825 |
> | SERVER AGGREGATE INTO SINGLE ROW
>   | 625043899   | 332170 | 150792825 |
> +---+-++---+
> select count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 0 |
> +---+
> {noformat}
> Using data table
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-+++
> |   PLAN  
>  | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO_TS   |
> +--+-+++
> | CLIENT 2-CHUNK 332151 ROWS 438492470 BYTES PARALLEL 1-WAY FULL SCAN OVER 
> TABLE_T  | 438492470   | 332151 | 1507928257617  |
> | SERVER FILTER BY FIRST KEY ONLY 
>  | 438492470   | 332151 | 1507928257617  |
> | SERVER AGGREGATE INTO SINGLE ROW
>  | 438492470   | 332151 | 1507928257617  |
> +--+-+++
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 14|
> +---+
> {noformat}
> Without stats available, results are correct:
> {noformat}
> explain select /*+NO_INDEX*/ count(*) from TABLE_T;
> +--+-++--+
> | PLAN | 
> EST_BYTES_READ  | EST_ROWS_READ  | EST_INFO_TS  |
> +--+-++--+
> | CLIENT 2-CHUNK PARALLEL 1-WAY FULL SCAN OVER TABLE_T  | null| 
> null   | null |
> | SERVER FILTER BY FIRST KEY ONLY  | null 
>| null   | null |
> | SERVER AGGREGATE INTO SINGLE ROW | null 
>| null   | null |
> +--+-++--+
> select /*+NO_INDEX*/ count(*) from TABLE_T;
> +---+
> | COUNT(1)  |
> +---+
> | 27|
> +---+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (PHOENIX-4287) Incorrect aggregate query results when stats are disable for parallelization

2017-10-13 Thread Mujtaba Chohan (JIRA)
Mujtaba Chohan created PHOENIX-4287:
---

 Summary: Incorrect aggregate query results when stats are disable 
for parallelization
 Key: PHOENIX-4287
 URL: https://issues.apache.org/jira/browse/PHOENIX-4287
 Project: Phoenix
  Issue Type: Bug
Affects Versions: 4.12.0
 Environment: HBase 1.3.1
Reporter: Mujtaba Chohan


With {{phoenix.use.stats.parallelization}} set to {{false}}, aggregate query 
returns incorrect results when stats are available.

With local index and stats disabled for parallelization:
{noformat}
explain select count(*) from TABLE_T;
+---+-++---+
| PLAN  
| EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO |
+---+-++---+
| CLIENT 0-CHUNK 332170 ROWS 625043899 BYTES PARALLEL 0-WAY RANGE SCAN OVER 
TABLE_T [1]  | 625043899   | 332170 | 150792825 |
| SERVER FILTER BY FIRST KEY ONLY   
| 625043899   | 332170 | 150792825 |
| SERVER AGGREGATE INTO SINGLE ROW  
| 625043899   | 332170 | 150792825 |
+---+-++---+

select count(*) from TABLE_T;
+---+
| COUNT(1)  |
+---+
| 0 |
+---+
{noformat}

Using data table
{noformat}
explain select /*+NO_INDEX*/ count(*) from TABLE_T;
+--+-+++
|   PLAN
   | EST_BYTES_READ  | EST_ROWS_READ  |  EST_INFO_TS   |
+--+-+++
| CLIENT 2-CHUNK 332151 ROWS 438492470 BYTES PARALLEL 1-WAY FULL SCAN OVER 
TABLE_T  | 438492470   | 332151 | 1507928257617  |
| SERVER FILTER BY FIRST KEY ONLY   
   | 438492470   | 332151 | 1507928257617  |
| SERVER AGGREGATE INTO SINGLE ROW  
   | 438492470   | 332151 | 1507928257617  |
+--+-+++

select /*+NO_INDEX*/ count(*) from TABLE_T;
+---+
| COUNT(1)  |
+---+
| 14|
+---+
{noformat}

Without stats available, results are correct:
{noformat}
explain select /*+NO_INDEX*/ count(*) from TABLE_T;
+--+-++--+
| PLAN | 
EST_BYTES_READ  | EST_ROWS_READ  | EST_INFO_TS  |
+--+-++--+
| CLIENT 2-CHUNK PARALLEL 1-WAY FULL SCAN OVER TABLE_T  | null| 
null   | null |
| SERVER FILTER BY FIRST KEY ONLY  | null   
 | null   | null |
| SERVER AGGREGATE INTO SINGLE ROW | null   
 | null   | null |
+--+-++--+
select /*+NO_INDEX*/ count(*) from TABLE_T;
+---+
| COUNT(1)  |
+---+
| 27|
+---+
{noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (PHOENIX-4280) Delete doesn't work when immutable indexes are in building state

2017-10-13 Thread James Taylor (JIRA)

 [ 
https://issues.apache.org/jira/browse/PHOENIX-4280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

James Taylor updated PHOENIX-4280:
--
Attachment: PHOENIX-4280_wip.patch

> Delete doesn't work when immutable indexes are in building state
> 
>
> Key: PHOENIX-4280
> URL: https://issues.apache.org/jira/browse/PHOENIX-4280
> Project: Phoenix
>  Issue Type: Bug
>Reporter: James Taylor
>Assignee: James Taylor
> Attachments: PHOENIX-4280_wip.patch
>
>
> We don't generate the Delete mutations correctly in this case as we assume 
> we're scanning over the index table.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PHOENIX-4242) Fix Indexer post-compact hook logging of NPE and TableNotFound

2017-10-13 Thread James Taylor (JIRA)

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

James Taylor commented on PHOENIX-4242:
---

My recommendation would be to have a separate JIRA for dealing with indexes on 
views. Let's just have this one prevent the NPE and TableNotFoundException.

> Fix Indexer post-compact hook logging of NPE and TableNotFound
> --
>
> Key: PHOENIX-4242
> URL: https://issues.apache.org/jira/browse/PHOENIX-4242
> Project: Phoenix
>  Issue Type: Bug
>Affects Versions: 4.12.0
>Reporter: Vincent Poon
>Assignee: Vincent Poon
> Attachments: PHOENIX-4242.v2.master.patch, 
> PHOENIX-4242.v3.master.patch, PHOENIX-4747.v1.master.patch
>
>
> The post-compact hook in the Indexer seems to log extraneous log messages 
> indicating NPE or TableNotFound.  The TableNotFound exceptions seem to 
> indicate actual table names prefixed with MERGE or RESTORE, and sometimes 
> suffixed with a digit, so perhaps these are views or something similar.
> Examples:
> 2017-09-28 13:35:03,118 WARN  [ctions-1506410238599] index.Indexer - Unable 
> to permanently disable indexes being partially rebuild for SYSTEM.SEQUENCE
> java.lang.NullPointerException
> 2017-09-28 10:20:56,406 WARN  [ctions-1506410238415] index.Indexer - Unable 
> to permanently disable indexes being partially rebuild for 
> MERGE_PLATFORM_ENTITY.PLATFORM_IMMUTABLE_ENTITY_DATA2
> org.apache.phoenix.schema.TableNotFoundException: ERROR 1012 (42M03): Table 
> undefined. tableName=MERGE_PLATFORM_ENTITY.PLATFORM_IMMUTABLE_ENTITY_DATA2



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (PHOENIX-4237) Allow sorting on (Java) collation keys for non-English locales

2017-10-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on PHOENIX-4237:
-

Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144620511
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: 

[GitHub] phoenix pull request #275: PHOENIX-4237: Add function to calculate Java coll...

2017-10-13 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144620511
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: 
%d, decomposition: %d], Special-Upper-Case: %s",
+   collator.getStrength(), 
collator.getDecomposition(), BooleanUtils.isTrue(useSpecialUpperCaseCollator)));
+   }
+   

[jira] [Commented] (PHOENIX-4237) Allow sorting on (Java) collation keys for non-English locales

2017-10-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on PHOENIX-4237:
-

Github user shehzaadn-vd commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144604412
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: 

[GitHub] phoenix pull request #275: PHOENIX-4237: Add function to calculate Java coll...

2017-10-13 Thread shehzaadn-vd
Github user shehzaadn-vd commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144604412
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: 
%d, decomposition: %d], Special-Upper-Case: %s",
+   collator.getStrength(), 
collator.getDecomposition(), BooleanUtils.isTrue(useSpecialUpperCaseCollator)));
+   }
+   

[jira] [Commented] (PHOENIX-4237) Allow sorting on (Java) collation keys for non-English locales

2017-10-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on PHOENIX-4237:
-

Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144600094
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: 

[GitHub] phoenix pull request #275: PHOENIX-4237: Add function to calculate Java coll...

2017-10-13 Thread JamesRTaylor
Github user JamesRTaylor commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144600094
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: 
%d, decomposition: %d], Special-Upper-Case: %s",
+   collator.getStrength(), 
collator.getDecomposition(), BooleanUtils.isTrue(useSpecialUpperCaseCollator)));
+   }
+   

Spark dataframe returning only structure when connected to Phoenix query server

2017-10-13 Thread Anjan K
Hi All,

I am connecting to hbase ( ver 1.2) via phoenix (4.11)  queryserver from
Spark 2.2.0, but the dataframe is returning the only table structure with
empty rows thoug data is present in table.

Here is the code I am using to connect to queryserver.



// ---jar phoenix-4.11.0-HBase-1.2-thin-client.jar

val prop = new java.util.Properties

prop.setProperty("driver", "org.apache.phoenix.queryserver.client.Driver")

val url = "jdbc:phoenix:thin:url=http://localhost:8765;serialization
=PROTOBUF"

val d1 = spark.sqlContext.read.jdbc(url,"TABLE1",prop)

d1.show()



Can anyone please help me in solving this issue. Thanks in advance


Re: [DISCUSS] Road to HBase 2.0.0

2017-10-13 Thread Josh Elser

Thanks, Anoop!

I know Sergey, Ankit, and Rajeshbabu have been looking at this already.

While tracking it is good, I think we still need to come up with a plan 
for where we're going to put that new code in Phoenix :)


On 10/13/17 6:58 AM, Anoop John wrote:

Thanks for bringing this up.  I was abt to ask this.  Ya the CP
framework itself and the CP exposed interfaces (Like
RegionServerServices, Region etc) are undergoing a big change for for
HBase 2.0..  I did  a look at some of the usages of the Phoenix
exposed interfaces/classes.  There are some items for fix.   Was
thinking to raise an umbrella issue once we have a plan for the
version based on HBase 2.0

-Anoop-

On Thu, Oct 12, 2017 at 3:30 AM, Josh Elser  wrote:

Since 4.12.0 is out and we have the concurrent discussions about the 0.98,
1.1, and 1.2 HBase branches, do folks have a vision of how we get to HBase
2.0.0?

The lack of chatter is pretty obvious that the Calcite work (the previous
impetus for Phoenix 5) has slowed. Once we get to an HBase 2.0.0-alpha4,
coprocessor API should stabilize and give us a point against which we can
start Phoenix work.

Should a release of Phoenix that supports HBase 2.0 be worthy of the Phoenix
5.0 label, or should we stick to the 4.x numbering? Given the breaking
changes going into HBase 2.0 and James' previous -1 to shim-layers for 0.98,
do we see the same for an HBase 2.0 branch or is HBase 1.x/2.x a different
beast? I can see pros/cons for both sides.

- Josh


Re: [DISCUSS] Road to HBase 2.0.0

2017-10-13 Thread Anoop John
Thanks for bringing this up.  I was abt to ask this.  Ya the CP
framework itself and the CP exposed interfaces (Like
RegionServerServices, Region etc) are undergoing a big change for for
HBase 2.0..  I did  a look at some of the usages of the Phoenix
exposed interfaces/classes.  There are some items for fix.   Was
thinking to raise an umbrella issue once we have a plan for the
version based on HBase 2.0

-Anoop-

On Thu, Oct 12, 2017 at 3:30 AM, Josh Elser  wrote:
> Since 4.12.0 is out and we have the concurrent discussions about the 0.98,
> 1.1, and 1.2 HBase branches, do folks have a vision of how we get to HBase
> 2.0.0?
>
> The lack of chatter is pretty obvious that the Calcite work (the previous
> impetus for Phoenix 5) has slowed. Once we get to an HBase 2.0.0-alpha4,
> coprocessor API should stabilize and give us a point against which we can
> start Phoenix work.
>
> Should a release of Phoenix that supports HBase 2.0 be worthy of the Phoenix
> 5.0 label, or should we stick to the 4.x numbering? Given the breaking
> changes going into HBase 2.0 and James' previous -1 to shim-layers for 0.98,
> do we see the same for an HBase 2.0 branch or is HBase 1.x/2.x a different
> beast? I can see pros/cons for both sides.
>
> - Josh


[jira] [Commented] (PHOENIX-4198) Remove the need for users to have access to the Phoenix SYSTEM tables to create tables

2017-10-13 Thread Ankit Singhal (JIRA)

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

Ankit Singhal commented on PHOENIX-4198:


Ping [~giacomotaylor] and [~tdsilva]

> Remove the need for users to have access to the Phoenix SYSTEM tables to 
> create tables
> --
>
> Key: PHOENIX-4198
> URL: https://issues.apache.org/jira/browse/PHOENIX-4198
> Project: Phoenix
>  Issue Type: Bug
>Reporter: Ankit Singhal
>Assignee: Ankit Singhal
>  Labels: namespaces, security
> Fix For: 4.13.0
>
> Attachments: PHOENIX-4198.patch, PHOENIX-4198_v2.patch, 
> PHOENIX-4198_v3.patch, PHOENIX-4198_v4.patch
>
>
> Problem statement:-
> A user who doesn't have access to a table should also not be able to modify  
> Phoenix Metadata. Currently, every user required to have a write permission 
> to SYSTEM tables which is a security concern as they can 
> create/alter/drop/corrupt meta data of any other table without proper access 
> to the corresponding physical tables.
> [~devaraj] recommended a solution as below.
> 1. A coprocessor endpoint would be implemented and all write accesses to the 
> catalog table would have to necessarily go through that. The 'hbase' user 
> would own that table. Today, there is MetaDataEndpointImpl that's run on the 
> RS where the catalog is hosted, and that could be enhanced to serve the 
> purpose we need.
> 2. The regionserver hosting the catalog table would do the needful for all 
> catalog updates - creating the mutations as needed, that is.
> 3. The coprocessor endpoint could use Ranger to do necessary authorization 
> checks before updating the catalog table. So for example, if a user doesn't 
> have authorization to create a table in a certain namespace, or update the 
> schema, etc., it can reject such requests outright. Only after successful 
> validations, does it perform the operations (physical operations to do with 
> creating the table, and updating the catalog table with the necessary 
> mutations).
> 4. In essence, the code that implements dealing with DDLs, would be hosted in 
> the catalog table endpoint. The client code would be really thin, and it 
> would just invoke the endpoint with the necessary info. The additional thing 
> that needs to be done in the endpoint is the validation of authorization to 
> prevent unauthorized users from making changes to someone else's 
> tables/schemas/etc. For example, one should be able to create a view on a 
> table if he has read access on the base table. That mutation on the catalog 
> table would be permitted. For changing the schema (adding a new column for 
> example), the said user would need write permission on the table... etc etc.
> Thanks [~elserj] for the write-up.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] phoenix pull request #275: PHOENIX-4237: Add function to calculate Java coll...

2017-10-13 Thread snakhoda-sfdc
Github user snakhoda-sfdc commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144483837
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: 
%d, decomposition: %d], Special-Upper-Case: %s",
+   collator.getStrength(), 
collator.getDecomposition(), BooleanUtils.isTrue(useSpecialUpperCaseCollator)));
+   }
+  

[jira] [Commented] (PHOENIX-4237) Allow sorting on (Java) collation keys for non-English locales

2017-10-13 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on PHOENIX-4237:
-

Github user snakhoda-sfdc commented on a diff in the pull request:

https://github.com/apache/phoenix/pull/275#discussion_r144483837
  
--- Diff: 
phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 ---
@@ -0,0 +1,233 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PIntegerArray;
+import org.apache.phoenix.schema.types.PUnsignedIntArray;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+import com.ibm.icu.impl.jdkadapter.CollatorICU;
+import com.ibm.icu.util.ULocale;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string 
based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * It uses the open-source grammaticus and i18n packages to obtain the 
collators
+ * it needs.
+ * 
+ * @author snakhoda
+ *
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args 
= {
+   // input string
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+   // ISO Code for Locale
+   @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+   // whether to use special upper case collator
+   @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+   // collator strength
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+   // collator decomposition
+   @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+   private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+   public static final String NAME = "COLLKEY";
+
+   public CollationKeyFunction() {
+   }
+
+   public CollationKeyFunction(List children) throws 
SQLException {
+   super(children);
+   }
+
+   @Override
+   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+   try {
+   String inputValue = getInputValue(tuple, ptr);
+   String localeISOCode = getLocaleISOCode(tuple, ptr);
+   Boolean useSpecialUpperCaseCollator = 
getUseSpecialUpperCaseCollator(tuple, ptr);
+   Integer collatorStrength = getCollatorStrength(tuple, 
ptr);
+   Integer collatorDecomposition = 
getCollatorDecomposition(tuple, ptr);
+
+   Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+   
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Locale: " + 
locale.toLanguageTag()));
+   }
+   
+   LinguisticSort linguisticSort = 
LinguisticSort.get(locale);
+
+   Collator collator = 
BooleanUtils.isTrue(useSpecialUpperCaseCollator)
+   ? 
linguisticSort.getUpperCaseCollator(false) : linguisticSort.getCollator();
+
+   if (collatorStrength != null) {
+   collator.setStrength(collatorStrength);
+   }
+
+   if (collatorDecomposition != null) {
+   
collator.setDecomposition(collatorDecomposition);
+   }
+
+   if(LOG.isDebugEnabled()) {
+   LOG.debug(String.format("Collator: [strength: