You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jonathan Ellis (JIRA)" <ji...@apache.org> on 2011/02/06 03:18:30 UTC

[jira] Issue Comment Edited: (CASSANDRA-1278) Make bulk loading into Cassandra less crappy, more pluggable

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

Jonathan Ellis edited comment on CASSANDRA-1278 at 2/6/11 2:17 AM:
-------------------------------------------------------------------

Thinking about this some more, I think we can really simplify it from a client perspective.

We could implement the Thrift Cassandra interface (the interface implemented by CassandraServer) in a bulk loader server.  ("Server" in that thrift clients connect to it, but it would run on client machines, not Cassandra nodes.)  

Writes would be turned into streaming, serialized-byte-streams by using Memtable + sort.  We would keep Memtable-per-replica-range, so the actual Cassandra node doesn't need to deserialize to potentially forward.  (Obviously we would not support any read operations.)

This approach would yield _zero_ need for new work on the client side -- you can use Hector, Pycassa, Aquiles, whatever, and normal batch_mutate could be turned into bulk load streams.

The one change we'd need on the client side would be a batch_complete call to say "we're done, now build 2ary indexes."  (per-sstable bloom + primary index can be built in parallel w/ the load, the way StreamIn currently does.)

Again, we could probably update the StreamIn/StreamOut interface to handle the bulkload daemon -> Cassandra traffice.  It _may_ be simpler to create a new api but my guess is not.

      was (Author: jbellis):
    Thinking about this some more, I think we can really simplify it from a client perspective.

We could implement the Thrift Cassandra interface (the interface implemented by CassandraServer) but writes would be turned into streaming, serialized-byte-streams (by using Memtable + sort).  We would keep Memtable-per-replica-range, so the actual Cassandra node doesn't need to deserialize to potentially forward.  (Obviously we would not support any read operations.)

Then there is _zero_ need for new work on the client side -- you can use Hector, Pycassa, Aquiles, whatever.

Well, almost zero: we'd need a batch_complete call to say "we're done, now build 2ary indexes."  (per-sstable bloom + primary index can be built in parallel w/ the load, the way StreamIn currently does.)

Again, we could probably update the StreamIn/StreamOut interface to handle this.  It _may_ be simpler to create a new api but my guess is not.
  
> Make bulk loading into Cassandra less crappy, more pluggable
> ------------------------------------------------------------
>
>                 Key: CASSANDRA-1278
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1278
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Tools
>            Reporter: Jeremy Hanna
>            Assignee: Matthew F. Dennis
>             Fix For: 0.7.2
>
>         Attachments: 1278-cassandra-0.7.txt
>
>   Original Estimate: 40h
>          Time Spent: 40h 40m
>  Remaining Estimate: 0h
>
> Currently bulk loading into Cassandra is a black art.  People are either directed to just do it responsibly with thrift or a higher level client, or they have to explore the contrib/bmt example - http://wiki.apache.org/cassandra/BinaryMemtable  That contrib module requires delving into the code to find out how it works and then applying it to the given problem.  Using either method, the user also needs to keep in mind that overloading the cluster is possible - which will hopefully be addressed in CASSANDRA-685
> This improvement would be to create a contrib module or set of documents dealing with bulk loading.  Perhaps it could include code in the Core to make it more pluggable for external clients of different types.
> It is just that this is something that many that are new to Cassandra need to do - bulk load their data into Cassandra.

-- 
This message is automatically generated by JIRA.
-
For more information on JIRA, see: http://www.atlassian.com/software/jira