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

Geoffrey Yu updated CASSANDRA-12367:
------------------------------------
    Attachment: 12367-trunk-v2.txt

I've attached another patch that adds in a new statement to CQL as described in 
the ticket for some early feedback on the approach. It's implemented as a new 
statement since the semantics of what it is meant to do did not fit in well 
with the existing {{SELECT}} statement.

{code}
cqlsh> SELECT SIZE FROM demo.test WHERE type = 'person';

 endpoint  | size (bytes)
-----------+--------------
 127.0.0.2 |          338
 127.0.0.3 |          338

(2 rows)
{code}

The statement needs to be restricted to a single partition, and returns results 
based on the consistency level (here it was {{ALL}} on a keyspace with RF=2).

{quote}
Could we use SSTableReader.getScanner(Range<Token> range, ...) instead of 
scanning all the partitions in the sstable? We would need to create the range 
so that it includes the token requested but I think it should save us some time 
by seeking to the correct position directly.
{quote}

Using {{SSTableReader.getScanner(Range<Token> range, ...)}} makes sense. Is 
there a recommended approach for creating a small {{Range<Token>}} that will 
wrap the requested token? For a {{LongToken}} it seems straightforward to just 
decrease the token in value slightly to create a range, but I'm not quite sure 
what a reasonable approach might look like for all the different types of 
tokens.

> 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