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

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

bq. No support for supercolumns?

Wow. Good catch. I've added test tests for this as well.

bq. it would be more clear if observeColumnsInSSTable took a CFMetaData object 
instead of a CF, to get a serializer from.

I've added a helper method CFMetaData.getColumnSerializer() to do this.

bq. nit: SSTMC.setMaxTimestamp would be more accurately named updateMaxTimestamp

Makes sense.

bq. IMO SSTM deserialize versioning logic would be clearer if it were all in 
SSTMSerializer instead of split between that and openFromDescriptor.

Makes sense.

bq. Suggest adding a comment that SSTableWriter.append(AbstractCompactedRow 
row) deliberately avoids calling updateMaxTimestamp b/c otherwise we'd have to 
deserialize EchoedRow.

Sounds good.

bq. where is the max-timestamp-of-compacted-sstables logic? I didn't notice it.

I put this in ColumnFamilyStore.createCompactionWriter():

{code}
public SSTableWriter createCompactionWriter(long estimatedRows, String 
location, Collection<SSTableReader> sstables) throws IOException
{    
    ReplayPosition rp = ReplayPosition.getReplayPosition(sstables);
    SSTableMetadata.Collector sstableMetadataCollector = 
SSTableMetadata.createCollector().replayPosition(rp);

    // get the max timestamp of the precompacted sstables
    for (SSTableReader sstable : sstables)
        sstableMetadataCollector.updateMaxTimestamp(sstable.getMaxTimestamp());

    return new SSTableWriter(getTempSSTablePath(location), estimatedRows, 
metadata, partitioner, sstableMetadataCollector);
}
{code}

bq. nit: renaming SSTableWriter.writeMetadata feels gratuitous

I renamed it back to writeMetadata.

bq. nit: prefer initializing fields that don't need constructor parameters, at 
declaration time (looking at RowIndexer.sstMC)

Makes sense.


> 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: 
> 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 
> 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

        

Reply via email to