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

Tyler Hobbs commented on CASSANDRA-12420:
-----------------------------------------

You are correct that this wouldn't break compatibility with other 2.1.x nodes, 
but it could cause problems during upgrades to 3.x.  In 3.x, another optional 
field (remainingInPartition) has already been added to the end of the 
PagingState serialization format.  Deserialization of the PagingState is only 
conditional on the native protocol version, not on the Cassandra version of 
other nodes in the cluster, so we can't safely introduce this change.

In CASSANDRA-6706, the decision was made to continue returning duplicate 
results in 2.1 when there are duplicate {{IN}} values in order to not make a 
(potentially) breaking change in a bugfix release.  However, this ticket 
represents a pretty big motivation to change that even in 2.1.  So, I'm 
thinking that we should go ahead and make 2.1 behave like 2.2 and 3.x and not 
return duplicate results in order to avoid this.

[~blerer] do you agree with the above?

> Duplicated Key in IN clause with a small fetch size will run forever
> --------------------------------------------------------------------
>
>                 Key: CASSANDRA-12420
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12420
>             Project: Cassandra
>          Issue Type: Bug
>          Components: CQL
>         Environment: cassandra 2.1.14, driver 2.1.7.1
>            Reporter: ZhaoYang
>            Assignee: ZhaoYang
>             Fix For: 2.1.x
>
>         Attachments: CASSANDRA-12420.patch
>
>
> This can be easily reproduced and fetch size is smaller than the correct 
> number of rows.
> A table has 2 partition key, 1 clustering key, 1 column.
> >        Select select = QueryBuilder.select().from("ks", "cf");
> >        select.where().and(QueryBuilder.eq("a", 1));
> >        select.where().and(QueryBuilder.in("b", Arrays.asList(1, 1, 1)));
> >        select.setFetchSize(5);
> Now we put a distinct method in client side to eliminate the duplicated key, 
> but it's better to fix inside Cassandra.



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

Reply via email to