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

Sylvain Lebresne commented on CASSANDRA-10606:
----------------------------------------------

I don't think that's what we should do. We want the internal storage engine not 
to care about compact tables are much as possible: the use of statics for 
compact tables should be and stay a higher level concern and as far as 
{{AbstractBTreePartition.rowCount()}} is concerned, counting the static row 
_only_ if there is no other rows is imo a weird contract. I'll note in 
particular that this method is used in a number of place for some data 
structure sizing and this patch would actually break those places.

Why do you think this is a problem btw? (I'll note that all counting that is 
CQL related uses {{DataLimits}} which takes that kind of subtlety into account) 



> AbstractBTreePartition.rowCount() return the wrong number of rows for compact 
> tables
> ------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-10606
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10606
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Benjamin Lerer
>            Assignee: Benjamin Lerer
>             Fix For: 3.0.0
>
>         Attachments: 10606-3.0.txt
>
>
> For compact tables {{AbstractBTreePartition.rowCount()}} return the wrong 
> number of columns as it does not take into account static rows.



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

Reply via email to