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

Sylvain Lebresne commented on CASSANDRA-12367:
----------------------------------------------

bq. Are these changes similar to what you had in mind?

Yes, that's what I had in mind, thanks. A few remarks from eye-balling it:
* You can get the uncompressed length of a {{SSTableReader}} with the 
{{uncompressedLength()}} method, no need to open a scanner.
* You can get the sstables for a given table using 
{{ColumnFamilyStore#getLiveSSTables()}} (or 
{{ColumnFamilyStore#getSSTables(SSTableSet.CANONICAL)}} if you really want the 
canonical set, though that probably doesn't matter much here) rather than 
iterating over all sstables of the keyspace.
* Would be more consistent to reuse {{StorageService#getValidColumnFamilies()}} 
rather than re-inventing your own checking (namely 
{{validateKeyspaceTableCombination}}).
* Regarding the memtable, it makes sense to have the option to include it, but 
I think we should be consistent in what we sum. For sstables, what we use is 
the serialized size of the partition, so I think we should do the same for 
memtables, that is call 
{{UnfilteredRowIteratorSerializer.serializedSize(partition.unfilteredIterator())}}.
* Wonders if it wouldn't be more user friendly to return 0 if the key is not 
hosted on that replica (which will simply happen if we don't check anything). 
Genuine question though, I could see both options having advantages, so 
mentioning it for the sake of discussion.
* I'd maybe call the JMX call {{getSerializedPartitionSize}} (or even 
{{getSerializedPartitionSizeInBytes}}) so it's a bit more explicit.


> Add an API to request the size of a CQL partition
> -------------------------------------------------
>
>                 Key: CASSANDRA-12367
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12367
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Geoffrey Yu
>            Assignee: Geoffrey Yu
>            Priority: Minor
>             Fix For: 3.x
>
>         Attachments: 12367-trunk-v2.txt, 12367-trunk.txt
>
>
> It would be useful to have an API that we could use to get the total 
> serialized size of a CQL partition, scoped by keyspace and table, on disk.



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

Reply via email to