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

Marcus Eriksson commented on CASSANDRA-7443:
--------------------------------------------

In general, this looks very good, few comments/thoughts;

I think we might want to force format-implementors to provide 'legacy' sstable 
scanners that generate OnDiskAtomIterators that work the same way as the 
current format. I see two cases here, either we compact sstables with the same 
format together, or we mix formats and these cases could be very different 
depending on how the formats look. I can, for example, imagine some formats 
being extremely efficient on merging Row Groups (columnar/parquet) but very 
slow at merging our legacy partitions, and we would need to merge partitions 
like that if we have one legacy sstable in the compacting sstables. Ie, what I 
think we need is something along the lines of an abstract 
'SSTableReader#getLegacyScanner', and then having the logics within the 
compaction code to use those scanners if we mix formats.

Small/nit code comments (note, didn't review the TestFormat as its mostly a 
PoC);
* Making RowIndexEntry generic needs to be propagated throughout the code, we 
now get more unchecked assignment warnings because of this.
* keepExistingFormat in compaction task should probably be handled some other 
way? should be possible to generate data, switch format and validate what we 
get?
* BigTableReader.SizeComparator() should reference 
SSTableReader.SizeComparator() in SizeTieredCompactionStrategy
* rename SSTableFormat#getCompactedRowWriter to just 'getCompactedRow'?
* SSTableFormat, Descriptor - wrong StringUtils import, should probably be 
org.apache.commons.lang3.StringUtils (thats what we use elsewhere anyway)


> SSTable Pluggability v2
> -----------------------
>
>                 Key: CASSANDRA-7443
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: T Jake Luciani
>            Assignee: T Jake Luciani
>             Fix For: 3.0
>
>         Attachments: 7443-refactor-v1.txt, 7443-testformat-v1.txt
>
>
> As part of a wider effort to improve the performance of our storage engine we 
> will need to support basic pluggability of the SSTable reader/writer. We 
> primarily need this to support the current SSTable format and new SSTable 
> format in the same version.  This will also let us encapsulate the changes in 
> a single layer vs forcing the whole engine to change at once.
> We previously discussed how to accomplish this in CASSANDRA-3067
>   



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

Reply via email to