cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Benedict (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine
Date Sun, 24 Nov 2013 23:29:36 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-5417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 0xFFFF?
	- 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 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 was sent by Atlassian JIRA
(v6.1#6144)

Mime
View raw message