cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alan Liang (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable
Date Tue, 14 Jun 2011 21:40:47 GMT

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

Alan Liang commented on CASSANDRA-2753:
---------------------------------------

There are basically 3 places where we need to track max timestamps:

1. Memtable flush
2. During compaction (we simply take the max timestamp already recorded for the sstables)
3. Streamed data (normal columns and counter columns)

The challenge here is to capture the max timestamp for newly streamed data. 

For non-counter streamed data, RowIndexer#doIndexing goes through the streamed data files
and simply updates the cache for the new rows. It iterates over the column families without
deserializing the columns. To capture max timestamp here, I actually deserialize the columns
from disk. This incurs more CPU but since it is already doing disk seeks when calling  deserializeFromSSTableNoColumns(),
the seek is less costly.

For counter streamed data, CommutativeRowIndexer#doIndexing actually creates new data files
from the streamed data files. It does this by building an AbstractCompactedRow which can be
either PreCompactedRow or LazilyCompactedRow. Collecting the max timestamp for PreCompactedRow
is easy since all the columns are in memory. For LazilyCompactedRow, the only place where
I can observe the max timestamp is during the #write method. Capturing the max timestamp is
obviously not ideal since it would introduce a side effect. Alternatively, I could capture
the max timestamp by deserializing the entire LazilyCompactedRow again but this obviously
would mean more IO/CPU.

So it looks like I have to capture the max timestamp inside #write.

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message