[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-11-25 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

Thanks for the review.

bq. 0xFF looks suspicious to me - should perhaps be 0x?

Correct, good catch (pushed a fix to the same branch as before).

bq. It looks like we're interning TimeUUIDType column identifiers

I don't think that's the case but we may have to be a little more precise here. 
 Bug notwithstanding, we only intern names that are part of the column_metadata 
in thrift parlance. Now thrift does let you declare a column_metadata that is a 
timeUUID (it's useless and probably nobody does it, but it's possible) but even 
in that case, we'll only intern the timeUUID that were declared. We do not 
intern dynamic names in particular if that is what you meant (contrarily to the 
current code I might add).

bq. CellName is confusing, as it isn't a name-of-a-cell, but a Cell-with-a-name

Well, actually no. It *is* a name-of-a-cell. At that point, I need to add that 
what we refer as 'cell' nowadays is what is currently the Column class.  
Obviously, the naming would be a lot more sensible if the Column class was 
actually named Cell, and we plan on changing that (CASSANDRA-6063), but I 
didn't wanted to include it in this patch-set because it's big enough as it is 
(CASSANDRA-6063 is marked 3.0 but I'm definitively for doing at least the 
Column-Cell renaming along with this patch (though I'd leave the rest of 
CASSANDRA-6063 for later)). Anyway, this also means than in your nits patch, 
I'd rather not do the cellNameFromByteBuffer-cellFromByteBuffer renaming.

bq. Compound(.*)CellName was also a little confusing (to me, at least), suggest 
either Prefixed/Clustered and (Blank)/Unclustered

Can't say I'm overjoy with Compound myself. However, Prefixed and/or Clustered 
doesn't really apply here (a simple cell name is not more or less 
prefixed/clustered than a compound one) so I don't think it's better. The 
difference between Simple and Compound (using the naming of the patch) is 
largely an implementation detail, namely whether the underlying encoding starts 
by 'number of components' or not (I would have loved not leaking the 
isCompound() method in the CellName interface in particular, to really nail the 
point that it's an implementation detail (for backward compatibility) but there 
still is a handful of places where we kind of need it so I've let it be).

{quote}
Remove Sparse from *SparseCellName - misleading, really it is just !Dense, so 
leave out Dense from the name
{quote}

At the risk of sounding obtuse, I've tried that initially and I kind of like it 
the way it is. Let me note that the Sparse naming doesn't really leak outside 
the 'composite' packages, most of the code just use the 'isDense' method of 
CellName and that's it. But as far as the implementations of CellName goes, I 
think SimpleCellName would suggest it's somewhat a super-class of 
SimpleDenseCellName which it's not (see the point below too). We do have 4 
largely orthogonal layout and I like making that very explicit in the 
implementation names. Probably a matter of personal preference though. 

bq. I would possibly suggest removing CellName interface, and having 
SimpleSparseCellName be simply NamedCell

That would be wrong imo. There is no meaningful inheritance relations between 
the cell names implementations (each pair of implementations do share a number 
of characteristics, which is why there is a few Abstract classes to avoid code 
duplication, but at the concrete implementation level there is no inheritance 
relation). I also like having CellName being an interface because that allows 
to cleanly have a few specific implementations like the EMPTY one (and the 
FakeCellName from ColumnSlice, though that one is largely a hack so it's 
probably a bad example).

bq. Also, suggest renaming CompositeBound to one of BoundedComposite

Good idea.

bq. Changed toByteBuffer to always return a ByteBuffer that cannot affect the 
state of the callee, and modified callers that were using .duplicate() to no 
longer do so

I'd rather not. We have an implicit rule in the code base that a caller should 
never expect it can modify the state of a ByteBuffer and it should call 
duplicate() if it needs to (one goal being to avoid defensive duplicate() when 
it's not needed since most of the code is written in a way that don't modify 
the ByteBuffers anyway) so the patch is consistent with the code base in that 
respect. Besides, in this particular case, thrift queries will call 
toByteBuffer() all the time without needing the duplication so leaving it to 
the caller does save some.

bq. Following code comment in CompositeType appears to be incomplete

Definitively not the most clear comment but as far as I can tell it's not 
incomplete. Can you clarify what you 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-11-25 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-5417:
-

bq. Bug notwithstanding, we only intern names that are part of the 
column_metadata in thrift parlance

Good point, I misread this at the start of processing the patch, should have 
confirmed once I'd got a proper handle on it.

bq. Well, actually no. It is a name-of-a-cell. ...  the naming would be a lot 
more sensible if the Column class was actually named Cell, and we plan on 
changing that...

Well, it seems to me the nomenclature is a little confused because we're 
overloading the name with data. This is parcelled up in the Prefix business 
too. A Simple cell name isn't really prefixed/clustered (by the terminology of 
CQL), and is simply a name-of-a-cell, but when we're mungung data into the name 
to support our clustering behaviour it becomes (to me) a 
(part-of-a-)cell-with-a-name. These were simply suggestions for maybe 
clarifying though, after struggling to grasp it initially. Since I understand 
what's going on now, I won't lose much sleep over the naming, and I can now see 
where you're coming from. I certainly agree that it's used to identify a cell, 
so I'll leave it be.

Sparse in particular threw me, as I can't see a reason for the moniker, and in 
normal English usage it means more than 'not dense', but I won't lose any sleep 
over it either.

bq. There is no meaningful inheritance relations between the cell names 
implementations

This was definitely my most tentative suggestion, and I suggested it only for 
simplifying the class naming and because it didn't seem *too* ugly. Definitely 
happy to ignore it.

bq. Besides, in this particular case, thrift queries will call toByteBuffer() 
all the time without needing the duplication so leaving it to the caller does 
save some

Fair enough - seemed we had the opposite problem with new BBs created in 
Composite*, but since duplicate() will be almost free in almost all cases, it 
was a bit of a non-issue in the first place. Happy to stick with the current 
conventions.

bq. Definitively not the most clear comment but as far as I can tell it's not 
incomplete.

Well, e.g., it says we can select = 'a AND = 'a' both by using 'a'0. I 
assume the intention is to use the latter in a range, with an inclusive lower 
bound and exclusive upper bound. But it isn't clear.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
  Labels: performance
 Fix For: 2.1


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-11-24 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-5417:
-

Took me a while to really understand everything that was going on here, but 
whilst I wouldn't say everything is intuitive to someone without much prior 
knowledge, I think the patch addresses the concerns pretty well for this round 
of changes. Normalising the storage of the prefix would certainly make the code 
cleaner and more understandable in future, though. Have a couple of minor 
functional concerns, some general nits I've cleaned up in 
[iss-5417-3|https://github.com/belliottsmith/cassandra/iss-5417-v3] and some 
nomenclature suggestions:

Concerns:
- AbstractCompoundCellNameType.fromByteBuffer:  0xFF looks suspicious 
to me - should perhaps be 0x?
- It looks like we're interning TimeUUIDType column identifiers, which 
probably isn't safe?

Nomenclature Suggestions:

* CellName is confusing, as it isn't a name-of-a-cell, but a 
Cell-with-a-name. Prefer NamedCell.
* Compound(.*)CellName was also a little confusing (to me, at least), 
suggest either Prefixed/Clustered and (Blank)/Unclustered
* Remove Sparse from *SparseCellName - misleading, really it is just 
!Dense, so leave out Dense from the name.
* I would possibly suggest removing CellName interface, and having 
SimpleSparseCellName be simply NamedCell, which others override, to keep 
nomenclature cleaner still.
* In summary, suggest the following or some mix thereof:
** NamedPrefixedCell/NamedClusteredCell
** NamedPrefixedDenseCell/NamedClusteredDenseCell
** NamedCell
** NamedDenseCell
* Also, suggest renaming CompositeBound to one of BoundedComposite, 
CompositeBounded, or CompositeInequality, due to ambiguity of meaning for 
Bound vs Bounded

General Minor Nits:

- isSameCQL3RowThan = isSameCQL3RowAs
- minor code cleanups (and some unnecessary removal of unecessary type 
parameters)
- Changed toByteBuffer to always return a ByteBuffer that cannot affect 
the state of the callee, and modified callers that were using .duplicate() to 
no longer do so
- Following code comment in CompositeType appears to be incomplete, as 
it suggests different outcomes for same spec:

/*
 * Given the rules for eoc (end-of-component, see 
AbstractCompositeType.compare()),
 * We can select:
 *   - = 'a' by using 'a'0
 *   -  'a' by using 'a'-1
 *   - = 'a' by using 'a'1
 *   -  'a' by using 'a'1
 *   - = 'a' by using 'a'0
 */


 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
  Labels: performance
 Fix For: 2.1


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-05-29 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

bq. Should we push this to 2.1?

Yes. I'd need to get back to this and I will but there is more pressing issues 
to fix for 2.0. 

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-05-28 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-5417:
---

Should we push this to 2.1?

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread T Jake Luciani (JIRA)

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

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

I agree the dremel approach would make the on disk format more efficient for 
composites.  I also think there is some lower hanging fruit to be had if we 
simply cache the current composite compare results so it isn't so cpu intense.  
Especially for the compaction case.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-5417:
---

We'll have single-pass compaction done RSN (CASSANDRA-4180).  Or did you mean 
something else for caching results?

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

bq. What if we took the approach from CASSANDRA-674 / 
http://wiki.apache.org/cassandra/FileFormatDesignDoc, and had an explicit 
parent atom instead of implicit via prefix sharing?

Just to be clear we talk of the same thing, when I speak of prefix sharing I'm, 
as far as this ticket is concern, talking of in-memory sharing. That being 
said, yes, I do think we should move at some point to a file format à la 
dremel/CASSANDRA-674. But that's more of a follow up to this ticket imo, it's 
not something that ticket is trying to tackle in itself.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

bq. I've attached a screenshot of the current bottleneck if you want to take a 
look.

I'll admit that puzzle me a bit. Is that saying that the 
AbstractCellNameType.columnSerializer() method itself is being a hot spot? 
Cause that method is simple getter of a final field! But anyway, I'll try to do 
some read tests, see if I can reproduce that ~30% slowdown and who is 
responsible.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread T Jake Luciani (JIRA)

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

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

I can send you my test offline if that helps.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

Oh, if that's relatively standalone, then yes that would be awesome.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread T Jake Luciani (JIRA)

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

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

Sure, I'll post it with the dataset tonight.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread T Jake Luciani (JIRA)

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

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

The best thing todo is just give you the uncompacted sstables...

https://docs.google.com/file/d/0B4FSNkh7LrJCc040UTRKZFdtTVk/edit?usp=sharing


You should keep the uncompacted sstables around and reset after each test
The two scenarios I tested were:
  1. Time it takes to perform a major compaction (with and without patch)
  2. Latency of reads for reading across all uncompacted tables (with and 
without patch)

Here is the schema:

{code}
CREATE KEYSPACE mjff WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 1};

use mjff;
CREATE TABLE data (
name text,
type text,
date timestamp,
value double,
PRIMARY KEY(name,type,date)
) WITH COMPACT STORAGE;
{code}

The reader code is simple:

{code}
public class StressReads {

static int threadCount = 2;

public static String[] names = new 
String[]{APPLE,VIOLET,SUNFLOWER,ROSE,PEONY,ORCHID,ORANGE,MAPLE,LILLY,FLOX,DAISY,DAFODIL,CROCUS,CHERRY};
public static String[] types = new String[]{diffSecs,N.samples,
x.mean,x.absolue.deviation,x.standard.deviation,
y.mean,y.absolue.deviation,y.standard.deviation,
z.mean,z.absolue.deviation,z.standard.deviation};

static ThreadLocalCassandra.Client client = new 
ThreadLocalCassandra.Client() {
public Cassandra.Client initialValue() {
   try{
TTransport trans = new TFramedTransport(new 
TSocket(localhost,9160));
trans.open();

TProtocol prot = new TBinaryProtocol(trans);
Cassandra.Client client = new Cassandra.Client(prot);

client.set_keyspace(mjff);

   return client;
   }catch(Exception e){
throw new RuntimeException(err, e);
   }
}
};

static ExecutorService threadPool = 
Executors.newFixedThreadPool(threadCount);

static AtomicLong totalReads = new AtomicLong(0);
static long allReads = 0;
static int countSeconds = 0 ;

static Random rand = new Random();
public static void main(String[] args) throws InterruptedException {

for(int i=0; ithreadCount; i++) {
   threadPool.submit(new Runnable() {
   @Override
   public void run() {
   while(true){
StringBuffer sb = new StringBuffer();
   sb.append(Select value from data where name=');
   sb.append(names[rand.nextInt(names.length)]);
   sb.append(' and type=');
   sb.append(types[rand.nextInt(types.length)]);
   sb.append(' and date  '2012-03-01 00:00:00' LIMIT 
100);

   try {
CqlResult result = 
client.get().execute_cql3_query(ByteBufferUtil.bytes(sb.toString()), 
Compression.NONE, ConsistencyLevel.ONE);

totalReads.addAndGet(result.getRows().size());
   }catch(Exception e){
   e.printStackTrace();
   }
   }
   }
   });
}


while (true) {
Thread.sleep(1000);

long reads = totalReads.getAndSet(0);
   allReads += reads;
System.err.println(Read +reads+ per/sec, avg 
+allReads/++countSeconds);
}
}
}
{code}


 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-26 Thread T Jake Luciani (JIRA)

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

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

Oh and for the read test you should explicitly disable minor compactions

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-25 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-5417:
---

What if we took the approach from CASSANDRA-674 / 
http://wiki.apache.org/cassandra/FileFormatDesignDoc, and had an explicit 
parent atom instead of implicit via prefix sharing?  Seems like that matches 
better what we'd want to do with the actual bits on disk.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-21 Thread T Jake Luciani (JIRA)

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

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

I've posted a couple minor commits to 
https://github.com/tjake/cassandra/tree/5417
There was one failing test and one performance regression.

I ran some benchmarks to gauge the effectiveness of this change.
I created a COMPACT table with a few wide rows and created 1k sstables. I 
benchmarked the time it took to perform a major compaction (since that causes 
the most Composite comparisons). 

First the base 
{code}
INFO 20:54:41,504 Compacted 942 sstables to 
[/var/lib/cassandra/data/mjff/data/mjff-data-ja-963,].  1,004,572,909 bytes to 
999,124,671 (~99% of original) in 372,638ms = 2.557011MB/s.  947 total rows, 6 
unique.
{code}


With this change.
{code}
INFO 22:25:46,747 Compacted 942 sstables to 
[/var/lib/cassandra/data/mjff/data/mjff-data-ja-963,].  1,004,572,909 bytes to 
999,124,671 (~99% of original) in 341,939ms = 2.786578MB/s.  947 total rows, 6 
unique.
{code}

So a bit of a improvement (~10%).  


Then I tested the performance of reading as quickly as possible.  This 
unfortunately was slower (~30%)
I've attached a screenshot of the current bottleneck if you want to take a look.





 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-11 Thread T Jake Luciani (JIRA)

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

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

Sorry still looking at this.  I've been thinking about the names and I think 
isPacked should become isSimple (since simple is the opposite of composite).  I 
see it's used in CType and Composites.

The whole packed and composite thing needs to be fixed in a later ticket :)

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-09 Thread T Jake Luciani (JIRA)

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

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

I started looking at this and so far my comments/questions are:

  -Why do we need AbstractCompositeType now that we use AbstractCType?
  -Why are you using the word 'Dense' internally when CQL3 uses 'Compact'?  I'd 
prefer to see s/Dense/Compact/
 

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-09 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

bq. Why do we need AbstractCompositeType now that we use AbstractCType?

One reason is that people may use a CompositeType as column value validator for 
instance. Removing it would break those. And as of this patch, CompositeType is 
still used for the row key even if it's composite. On that last part, I do 
think we should replace it by CType internally at some point because that would 
make sense, but I felt the current patch was big enough currently that this 
could wait a followup ticket.

Another reason is that it's simpler for compatibility. The fact is, even if we 
don't use CompositeType internally, for compatibility sake we need to style 
parse and return stuffs like CompositeType(UTF8Type, IntegerType) in thrift 
CfDef and such. And more generally, there is a few of places in the code where 
it's just simpler to just convert the CType to an equivalent old style 
CompositeType (the cql2 has some of those for instance). Another more minor 
reason is that if someone is crazy enough to use a super column with a 
compositeType comparator or subcomparator today, we'll still use the old 
CompositeType. As for AbstractCompositeType specifically, it's also reused for 
DynamicCompositeType.

That being said, I think that if we do the follow up ticket I mention above of 
using CType for composite row keys to, then I think we can remove some bits 
like the CompositeType.Builder.

bq. Why are you using the word 'Dense' internally when CQL3 uses 'Compact'?

Because they are not equivalent :). I'll admit, this could just be a bad choice 
of terms, but I haven't found much better. But anyway, internally there is 2 
dimensions to the layouts used distinguished by CQL3 which are:
# whether the comparator is either composite or not. In the code I use 
isPacked() to mean not composite at times, though that's also a I haven't 
found better term.
# whether one of the component (the last one or 2nd to last if there is a 
collection) of the comparator stores the CQL3 column name. If that's the case, 
then the CF is not 'dense' (I might use sparse at times in that case), 
otherwise it is. Another more pragramatic to see that is to say that dense CFs 
are those where 1 CQL3 row == 1 internal cell.

Those are the distinction that are useful internally.

But while the 4 combinations of those 2 dimensions make sense, only one of them 
(non packed, non dense) is the default CQL3 layout. All the others are 
created in CQL3 using WITH COMPACT STORAGE. And so in particular, a COMPACT 
static CF, i.e. something like
{noformat}
CREATE TABLE foo {
  k int PRIMARY KEY,
  c1 int,
  c2 text,
  c3 double
} WITH COMPACT STORAGE
{noformat}
is *not* dense (but it is packed). One CQL3 row correspond to a group of 
internal cells.

I'm happy to change the term 'dense' internally if someone has a better 
suggestion, but using 'compact' would confuse things more than anything imo.

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-09 Thread T Jake Luciani (JIRA)

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

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

It seems like this patch cleans up a lot for the cql3 use case, but it is 
adding to the overall mess that we have from supporting all these different 
implementations.
The storage engine is too tied to the public APIs and CQL seems to just be 
pushing itself along side it now.

Things like CType.asAbstractType is obviously a problem and it makes me worry 
about the maintainability of this kind of change.  

If anything this patch doesn't go far enough since I don't see where/how this 
gets better.  
I'll keep absorbing for a few more days...





 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-09 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

bq. it is adding to the overall mess that we have from supporting all these 
different implementations

It was my hope that the code was at least encapsulating thing more cleanly. 

bq. The storage engine is too tied to the public APIs

I don't disagree, but at the same time we've made promises that we wouldn't 
break most of those public API.

bq. Things like CType.asAbstractType is obviously a problem and it makes me 
worry about the maintainability of this kind of change

Fair enough. But for what it's worth, things like asAbstractType don't seem 
like a big deal to me. There is a few shortcuts like that that the patch use to 
avoid rewriting all of Cassandra in one patch, but imo they are used in 
relative clear boundaries (i.e. between thrift/cql2 and the internal storage). 
At least, that was the intent :)

bq. If anything this patch doesn't go far enough since I don't see where/how 
this gets better

To be clear, I don't intend this patch to be an end. There is definitively more 
things that can be cleaned up and we can absolutely push things further. But at 
the same time, I'm not sure rewriting all the things in one go is going to be 
realistic.

I do think that the impedance mismatch between CQL3 and the storage engine is 
getting in our way, and I do think we should do something about it. This patch 
is a suggestion, which is trying to take an incrementalish approach. I'm open 
to other solution/modification though. But I'll also note that cleaning up 
CQL3 code is not the only reason for this ticket, encapsulating the composite 
nature of column names so we can optimize things later is another (and here 
again, I realize that the patch itself does not add the optimizations per se, 
but I was just not convinced that sealing myself in a cave for 2 months to 
implement all the optimization I can think of that make use of this ticket 
would be the best strategy).



 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-03 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

I've pushed some initial patch to 
https://github.com/pcmanus/cassandra/commits/5417. It's on top of 
CASSANDRA-5125 as said above (so only the 3 last commits are for this ticket).

I'll note right away that this patch mainly concern itself with the ground 
work, not with the different optimization I discuss in the description.  
Typically, it doesn't do any prefix sharing and a bit more work will be 
necessary to get that, but it makes it at least possible. Also, while the patch 
does clean up the CQL3 code quite a bit imo, I think there is more cleanups 
opportunity that are not taken just yet, but I feel that the patch is huge 
enough like that, so I'd rather keep that for later.

The patch passes the unit tests as well as (almost all) the cql dtests. For the 
cql dtests, due to some refactor, the patch introduce a small limitation that 
if you use a IN on the last clustering key, you have to include that last 
clustering key in the selected columns. This makes one test fail. We should 
definitively fix it, but as it happens, the solution for that is exactly the 
one needed for CASSANDRA-4911. So since that's a bit of a detail, and since 
this patch is already pretty big, I suggest leaving the fix of that remaining 
dtest to CASSANDRA-4911 and I'll take on myself to fix that latter ticket on 
2.0 timeframe.

I'll also note that this patch does bias the code a bit towards Composite, in 
the sense that for non-composite CF, we do allocate one more object per column 
(compare to without the patch). I haven't done much performance testing at that 
point however.


 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing 

[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-02 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-5417:
-

I have a patch almost done that I'll past soonish. I do note that said patch is 
on top of CASSANDRA-5125 because some of the refactor of that latter ticket 
made things simpler. So it someone feels like reviewing CASSANDRA-5125, that'll 
help :)

 Push composites support in the storage engine
 -

 Key: CASSANDRA-5417
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


 CompositeType happens to be very useful and is now widely used: CQL3 heavily 
 rely on it, and super columns are now using it too internally. Besides, 
 CompositeType has been advised as a replacement of super columns on the 
 thrift side for a while, so it's safe to assume that it's generally used 
 there too.
 CompositeType has initially been introduced as just another AbstractType.  
 Meaning that the storage engine has no nothing whatsoever of composites 
 being, well, composite. This has the following drawbacks:
 * Because internally a composite value is handled as just a ByteBuffer, we 
 end up doing a lot of extra work. Typically, each time we compare 2 composite 
 value, we end up deserializing the components (which, while it doesn't copy 
 data per-se because we just slice the global ByteBuffer, still waste some cpu 
 cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
 called *a lot*, this is likely not negligible.
 * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
 use of composites, and since it gets backs ByteBuffer from the internal 
 columns, it always have to check if it's actually a compositeType or not, and 
 then split it and pick the different parts it needs. It's only an API 
 problem, but having things exposed as composites directly would definitively 
 make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
 has a composite with only one component or a non-really-composite value, but 
 we still always distinguishes both cases.  Lastly, if we do expose composites 
 more directly internally, it's not a lot more work to internalize better 
 the different parts of the cell name that CQL3 uses (what's the clustering 
 key, what's the actuall CQL3 column name, what's the collection element), 
 making things cleaner. Last but not least, there is currently a bunch of 
 places where methods take a ByteBuffer as argument and it's hard to know 
 whether it expects a cell name or a CQL3 column name. This is pretty error 
 prone.
 * It makes it hard (or impossible) to do a number of performance 
 improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
 properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
 names are just one of the component in general). But we also miss 
 oportunities of sharing prefixes. If we were able to share prefixes of 
 composite names in memory we would 1) lower the memory footprint and 2) 
 potentially speed-up comparison (of the prefixes) by checking reference 
 equality first (also, doing prefix sharing on-disk, which is a separate 
 concern btw, might be easier to do if we do prefix sharing in memory).
 So I suggest pushing CompositeType support inside the storage engine. What I 
 mean by that concretely would be change the internal {{Column.name}} from 
 ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
 ByteBuffer. But in practice, we'd have a specific CellName implementation for 
 not-really-composite names, and the truly composite implementation will allow 
 some prefix sharing. From an external API however, nothing would change, we 
 would pack the composite as usual before sending it back to the client, but 
 at least internally, comparison won't have to deserialize the components 
 every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira