cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sylvain Lebresne (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-10707) Add support for Group By to Select statement
Date Tue, 01 Mar 2016 10:27:18 GMT

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

Sylvain Lebresne commented on CASSANDRA-10707:
----------------------------------------------

bq. The first point that needs discussing is that this patch changes the inter-node protocol
and this without bumping the protocol version

We had some offline discussion about this, and the consensus seems to be that we'll leave
it as is for this patch with just a fat warning in the NEWS file that you shouldn't use {{GROUP
BY}} until you've fully upgraded. As said above, this is not perfect if someone don't follow
that arguably intuitive instruction but this'll do for this time. In the meantime, we'll fix
the protocol deserialization so it doesn't drop the connection if a message has more than
it expects, but just skips the message remainder. Longer term, we should introduce at least
major and minor versioning for the messaging protocol so we can deal with this in a better
way.

bq. the operation (filtering and ordering) commute. That's not really the case for {{ORDER
BY}} and {{GROUP BY}}.

Actually, I guess the grouping itself commutes, it's more the aggregation  that depend on
the order. So nevermind, I'm good with sticking to the SQL syntax.

bq. Having a GroupMaker implementation for normal queries simplify the code as the same algorythm
can be used for the 3 scenarios.

I read the code too quickly, sorry, but still, I meant to go the same route than for {{GroupSpecification.NO_GROUPING}}.
The naming is equally confusing imo and the code simplification is pretty detatable: we reference
that {{GroupMaker}} in {{Selection}} (only place where {{NO_GROUPING}} can be used I believe)
twice, so using some {{groupMaker != null}} won't make a big difference.

bq. Even if we allow functions the {{lastClustering}} will always be a the last clustering.

Fair enough, though I still feel like grouping it with the partition key in a {{GroupMaker.State}}
would be a tad cleaner. And at the very least, why use a {{ByteBuffer[]}} for the clustering
instead of {{Clustering}} which is a more explicit?

bq. I tried the approach that you suggested but without success

I'll try to have a look in the coming days because I do feel it would be cleaner and it ought
to be possible but ...

bq. Performing the modification outside of the {{CQLGroupByLimits}} is probably possible but
will force us to modify the {{DataLimits}} and {{QueryPager}} interfaces to expose the {{rowCount}}.

... I might be misunderstanding what this imply but that doesn't sound particularly bad to
me.


A few other remarks:
* In the news file, you have {{IN restrictions with only one element are now considered as
equality restrictions}}. What does that mean for the user?
* Could remove {{CFMetadaData.primaryKeyColumns()}} now that it's unused.
* The comment in {{DataLimits.CQLLimits.hasEnoughLiveData}} still misses some part, it used
to (and should) read {{Getting that precise _number forces_ us ...}}.
* Forgot that initially, but in {{DataLimits.CQLGroupByLimits.forPaging(int, ByteBuffer, int)}},
it's fishy to me that we use the partition key in parameters but reuse the pre-existing {{lastClustering}}.
If we're guaranteed than {{lastReturnedKey == lastPartitionKey}} then we should assert it
as it's not immediatly obvious, otherwise this is wrong.


> Add support for Group By to Select statement
> --------------------------------------------
>
>                 Key: CASSANDRA-10707
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10707
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: CQL
>            Reporter: Benjamin Lerer
>            Assignee: Benjamin Lerer
>
> Now that Cassandra support aggregate functions, it makes sense to support {{GROUP BY}}
on the {{SELECT}} statements.
> It should be possible to group either at the partition level or at the clustering column
level.
> {code}
> SELECT partitionKey, max(value) FROM myTable GROUP BY partitionKey;
> SELECT partitionKey, clustering0, clustering1, max(value) FROM myTable GROUP BY partitionKey,
clustering0, clustering1; 
> {code}



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

Mime
View raw message