[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-23 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-7443:


+1

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-20 Thread T Jake Luciani (JIRA)

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

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

[~krummas] [~slebresne] I've rebased and incorporated your feedback here

https://github.com/tjake/cassandra/tree/7443-final

All public access to format types is removed and there is a single test 
property cassandra.test.sstableformatdevelopment added to allow us access to 
write tests that allow us to test things like compaction of a particular format 
(during development).  



 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread Marcus Eriksson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread T Jake Luciani (JIRA)

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

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

bq.  force format-implementors to provide 'legacy' sstable scanners that 
generate OnDiskAtomIterators that work the same way as the current format.

Not sure I follow, the iterator must still return data in the same order 
regardless of the format.  Out of Comparator order will throw an exception in 
the ColumnFamily.  We could add an explicit check lower down in the code but 
then it's redundant IMO.

bq. keepExistingFormat in compaction task should probably be handled some other 
way? should be possible to generate data, switch format and validate what we 
get?

I only added this for A test case where you stream in data from a new format 
and compact it in the same format (without globally requiring all sstables to 
be written in the same format)
Outside of tests it should not be allowed. I can add a @VisibleForTesting and a 
better comment.

I'll fix the other nits and rebase

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-7443:
-

At the risk of repeating myself, I'm fine with having better abstractions 
internally that make it easier to transition sstable format, but I'm still very 
opposed to anything more (and I know I'm not the only one). Concretely, I'm 
fine with the patch in general but I won't be happy unless anything user 
visible is removed, which means at least the yaml/config setting and the 
CQLSStableWriter.Builder method (and whatever else I've missed).

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread T Jake Luciani (JIRA)

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

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

bq.  I won't be happy unless anything user visible is removed

I also agree these should be removed since, as you mentioned, is not the intent 
of this patch.   I can hide those settings but internally we still need 
access to them as developers.

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-7443:


bq. the iterator must still return data in the same order regardless of the 
format.
right, but I'm imagining (perhaps prematurely) that some sstable formats could 
be compacted in a much more efficient way if we have a way of scanning sstables 
without that requirement during compaction

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread T Jake Luciani (JIRA)

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

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

Ah.  I would prefer to deal with that once we have the basics covered in the 
new format.

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-7443:
-

bq.  can hide those settings but internally we still need access to them as 
developers

Why? We've always transitioned sstables format by continuing to read old 
sstables and only writing new ones, and I'm fan of sticking to that. If we do 
so, the actual format used is hardcoded for write (for a given C* version) and 
based on the sstable for read. No reason to have a Config setting, even hidden. 
Note that if we ever change our mind on how we want to proceed, adding a hidden 
setting is pretty easy, but I'd rather not anticipate on stuffs we haven't 
agreed on yet.

bq. I'm imagining (perhaps prematurely) that some sstable formats could be 
compacted in a much more efficient way if we have a way

I do think that's premature. If we come up with a format that support that, we 
can always find the proper abstraction at that time.

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-7443:


sure!

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread T Jake Luciani (JIRA)

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

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

bq. Why? 

I'm literally talking about for purposes of developing the new format (for 
example the test format attached to this ticket).  We need a way to develop 
tests against only the new format and not the old. so we need access to writing 
a specific format.   We can technically do it without that but then we have a 
chicken and egg problem, we can't run the test harness without a fully 
functional sstable format.   Once the format is working I'm happy to remove 
those options but at the moment it's needed.

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-10-13 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-7443:
-

All I'm saying is that for tests, having (static) getter and setter in 
{{DatabaseDescriptor}} ought to be enough, no need to have it in {{Config}}. If 
that's what you meant by hidden setting, we're in agreement then.

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-09-02 Thread T Jake Luciani (JIRA)

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

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

github branch is https://github.com/tjake/cassandra/tree/7443-cleanup

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-08-04 Thread T Jake Luciani (JIRA)

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

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

One point to discuss is the upgrade path.  My vote is we should support a per 
node default for the SSTable format (what's currently implemented),  
This way a user can take a node and upgrade it to the new format before going 
forward.  

When a user does set the format to the non-default it will end up incrementally 
upgrading the sstables as they are compacted effectively the same as how the 
current sstable revisions work. He/she could also run upgradesstables.

I think making it a per node setting makes sense for users who want to opt-in 
initally similar to how we did vnodes and sstable compression.  Once the format 
is proved in battle tested we can either change the default format or remove 
the setting altogether.



 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-07-30 Thread T Jake Luciani (JIRA)

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

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

bq. Streaming may need a bit more work/thought - seems that compression format 
of sstables is coupled quite tightly with compression stream writer, and also 
assumes we can stream a single file range. Might not want to impose that 
requirement (we probably don't for 3.0 format - a set of ranges is more likely, 
but I would prefer to abstract the concept of a stream Chunk to be format 
specific anyway, to remove the coupling)

I'm working on this now and the simplest solution might be to write the stream 
to disk first and then use a RAF to process it.  If we use row groups you would 
still need to send all the data in the partition, the issue is that you can't 
process it sequentially.

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-07-30 Thread T Jake Luciani (JIRA)

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

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

Or like you said, make it a chunk, which in the case of the new format would be 
a rowgroup... hmmm

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-07-21 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-7443:
-

First impression is pretty positive - this is looking a lot cleaner than I 
might have hoped.

A few initial comments, obviously bearing in mind it's still in progress:

* I'm not super keen on using Class.getConstructor() for instantiating a 
reader/writer. Would prefer a factory method / class; using reflection for long 
type parameter lists worries me.
* Would prefer DatabaseDescriptor.getDefaultSSTableFormat, and permit overrides 
per Table
* LazilyCompactedRow is still tightly coupled with Big format (relates to 
AbstractCompactedRow as you mentioned)
* Streaming _may_ need a bit more work/thought - seems that compression format 
of sstables is coupled quite tightly with compression stream writer, and also 
assumes we can stream a single file range. Might not want to impose that 
requirement (we probably don't for 3.0 format - a set of ranges is more likely, 
but I would prefer to abstract the concept of a stream Chunk to be format 
specific anyway, to remove the coupling)

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-07-18 Thread T Jake Luciani (JIRA)

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

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

Problems;

Realizing the AbstractCompactedRow needs to come from the Format implementation 
since it currently is specific to the BigTable format

The on disk serializers are statically tied to the types so while you can just 
ignore them in the reader/writers it needs to be re-factored so they are tied 
to the format.

 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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-07-17 Thread T Jake Luciani (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-07-02 Thread T Jake Luciani (JIRA)

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

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

.bq I know its off topic anyway but...

Take a look at https://issues.apache.org/jira/browse/CASSANDRA-6602

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-07-01 Thread Edward Capriolo (JIRA)

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

Edward Capriolo commented on CASSANDRA-7443:


I know its off topic anyway but...

I would suggest that Sized and LevelDB compose what essentially are two 
pluggable storage engines, also both can be compressed,  so maybe that is 2.5. 

There are other ticktes around minimizing re-compaction for write once TTL only 
data. I do agree that having N storage engines would be a bad. But I do not 
feel like sized and leveled are the best fit for a variety of use cases. 

I have a column-family-per-month design to get around re compaction issues in a 
time series application. Many suggest that cassandra is the IDEAL timeseries 
database, but its current compaction system for level/sized does not seem like 
the perfect fit to me.


 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-7443:
-

A quick comment: we may need to make pluggability a bit higher in the stack 
than sstable reader/writer; CcollationController should probably be involved in 
this step. A related goal to keep in mind, whilst not the explicit goal of this 
ticket, is making replacing ColumnFamily with a new internal 
resultset/iterator abstraction a little easier. We should position ourselves to 
be able to tackle that after this (or one more follow up) ticket.

We will eventually want to make writes configurable before they hit memtables 
as well, though that can probably be bumped to a later ticket also.

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread Jason Brown (JIRA)

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

Jason Brown commented on CASSANDRA-7443:


Do we actually want to move the abstraction even higher? When I was 
experimenting with creating a pluggable storage engine, I started by splitting 
ColumnFamilyStore in half, as that's where a lot of things like compaction 
strategy, sstable, and such, begin to be exposed.

bq. We will eventually want to make writes configurable before they hit 
memtables as well

Yeah, this is definitely starting to sound like a pluggable storage engine :). 
Thus, that's why I think CFS is a good place place to start. 

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-7443:
-

We will have to dice up CFS somewhat no doubt, yes - as soon as we start 
messing with CC and Memtables, CFS is in the mix too. With some of these 
changes and compaction etc., I'm sort of tempted to wait until we have a new 
storage API (which may follow this ticket)... we need to try and do this 
piecemeal as it is going to be a difficult project. Let's make it minimally 
abstract, then iterate from there :)

Specifically for compaction, you can already drop a custom compaction strategy 
(which permits, e.g., turning it off), but we probably don't want to 
reimplement LCS/STCS etc. for each storage engine that needs them.

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread Jason Brown (JIRA)

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

Jason Brown commented on CASSANDRA-7443:


bq. we probably don't want to reimplement LCS/STCS etc. for each storage engine 
that needs them

Agreed, but not all storage engines will need compaction. I'm not sure it's as 
easy as a custom compaction strategy (which permits, e.g., turning it off), 
but I would need to think through it some more. However, storage engines should 
be able to freely pull in existing components like compaction strategies, 
current memtable, and the like. 


 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread T Jake Luciani (JIRA)

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

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

[~jasobrown]  Pluggable storage engines are out of scope for this work.  I 
think starting that high would be counter productive for the rest of the 
project (since it would be such an invasive change).  We can take that on as 
phase II once we prove out the storage format changes.

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-7443:
-

Before people get too deep on this, I just want to note that a priori, I have a 
relatively strong feeling against pluggable storage engine if it's an 
external feature (where there would be competing engines).  Because I think 
it would be a disservice to make to Cassandra in general. Multiple engines 
means that some users will pick the wrong one with generally little or no way 
to change, it complicates testing/maintenance and it can easily limit changes 
because that would broke the assumption that engine X or Y uses.

Please note however that I have no problem at all with cleaning up internal 
APIs so it's easier for us to experiment with changes/improvements, and if 
that's really just what we're talking about here, I'm +1. But if we're talking 
about providing some form of external guarantee on those API and making it 
official that people can bring their own storage engine, then be aware that I 
will need a serious amount of convincing.

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread Jason Brown (JIRA)

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

Jason Brown commented on CASSANDRA-7443:


Wow, alright, let me retract the storage engine notion from this ticket. I 
didn't mean to cloud [~tjake]'s original intent here. I guess I meant clean up 
internal APIs instead, and went a bit too far.

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)


[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2

2014-06-25 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-7443:
-

To clarify, and I hope put everyone at ease, I have no interest in explicitly 
supporting an officially guaranteed pluggable storage API - but in making it 
easy for us to swap out the implementation we do necessarily need to make it 
somewhat pluggable internally

 SSTable Pluggability v2
 ---

 Key: CASSANDRA-7443
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443
 Project: Cassandra
  Issue Type: Bug
  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)