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

T Jake Luciani commented on CASSANDRA-7443:
-------------------------------------------

I've posted my work so far here 
https://github.com/tjake/cassandra/tree/7443-plugsstable

In general it is still pretty rough and I need to clean up more but, the idea 
is formats would still require BloomFilter, Index and Data components.   Beyond 
that the data can be laid out however you want.   The readers now provide the 
OnDiskAtomIterator along with the ICompactionScanner so a format writer has 
total control over those.  But the rest of the APIs are left intact. 

I don't want to go too deep in this ticket since we should get this kind of 
change into the codebase then iterate more vs a big bang, but this should be 
enough to get started writing the new format.  

The approach I took was the following:

  * There is a system level setting to pick the default "format" the system 
will use to write files.
  * Added a "format" component to the file Descriptor class.  If missing it 
will use the existing legact format by default. 
  * Added a SSTableFormat class that defines the SSTableReader and 
SSTableWriter implementations as well as the "features" of the format (like 
hasNewStatsFile etc that used to be contained in the Descriptor class 
directly).  We can extend to toggle on different code paths while keeping the 
old logic intact.
  * There is a pre-defined list of formats we support (currently only the 
existing format, I'm calling it BIG since its Big literally and BigTable)
  * Re-wired the Reader and Writer creators so the classes are constructed 
based on the SSTableFormat information in the Descriptor
  * Abstracted the Writer api (See SSTableWriter)
  * Abstracted the Reader api (See SSTableReader)
  * Moved the current Format files to be prefixed with "Big" under the 
o.a.c.db.sstable.format.big namespace
  * Moved the iterators and scanners to be fetched only from the SStablereader 
API.  

All the test pass and I added code around the Streaming so it will Error if you 
try to stream a non legacy sstable to an old version node.



As for next steps,  I want to build a very differently implemented "TEST" 
format to make sure all the paths are covered and all the tests still pass.





> 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
>
>
> 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.2#6252)

Reply via email to