You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@cassandra.apache.org by Johan Oskarsson <jo...@oskarsson.nu> on 2009/09/01 21:48:09 UTC
Re: Cassandra + Hadoop + BMT
I have slapped together a basic Hadoop 0.18 CassandraOutputFormat based
on the code Chris put up.
Usage:
conf.setOutputKeyClass(RowColumn.class);
conf.setOutputValueClass(BytesWritable.class);
conf.setOutputFormat(CassandraOutputFormat.class);
conf.set(CassandraOutputFormat.CONF_COLUMN_FAMILY_NAME, "columnfamilyname");
conf.set(CassandraOutputFormat.CONF_KEYSPACE, "keyspacename");
DistributedCache.addCacheFile(new URI("uri_to_storage-conf.xml"), conf);
+ your job specific settings.
Then after the job run this method: CassandraOutputFormat.forceFlush
Source code here:
http://github.com/johanoskarsson/cassandraoutputformat/tree/master
Big thanks to Chris for figuring out the mystery that is BinaryMemtable
/Johan
Chris Goffinet wrote:
> Hi Guys
>
> This is long overdue but I have posted a very rough rough example (with
> Digg stuff removed) for getting BMT working with Cassandra. Patches are
> coming next up for the JIRA tickets. I'll try to get a more generic
> map/reduce job finished by end of the week that integrates Hive output.
>
> http://github.com/lenn0x/Cassandra-Hadoop-BMT/tree/master
>
> -Chris
Re: Cassandra + Hadoop + BMT
Posted by Jun Rao <ju...@almaden.ibm.com>.
Thanks, Johan.
I think you can simplify you code by using org.apache.cassandra.client
.RingCache (see test/unit/org.apache.cassandra.client.TestRingCache for
example).
Jun
IBM Almaden Research Center
K55/B1, 650 Harry Road, San Jose, CA 95120-6099
junrao@almaden.ibm.com
|------------>
| From: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|Johan Oskarsson <jo...@oskarsson.nu> |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|------------>
| To: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|cassandra-dev@incubator.apache.org |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|------------>
| Date: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|09/09/2009 02:49 AM |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|------------>
| Subject: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|Re: Cassandra + Hadoop + BMT |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
In my version of the code the storage endpoints are pulled out from a
seed node using the NodeProbe class and then put into the StorageService
using the updateTokenMetadata method.
See updateTokenMetadata in CassandraClient:
http://github.com/johanoskarsson/cassandraoutputformat/blob/dfa4dbf9b1bc81854b492af14536693002e19e52/src/java/fm/last/hadoop/mapred/CassandraClient.java
Granted it's not a perfect solution.
/Johan
Jun Rao wrote:
> I was trying to understand how the MapReduce job figures out where a row
> is located in a cassandra cluster and I saw the following code. Does
> this really work? To compute the proper endpoints, the StorageService
> needs to be started to obtain all tokens from other nodes through
> gossip. However, StorageService is not started in the MapReduce job.
>
> for (EndPoint endpoint :
> StorageService.instance().getReadStorageEndPoints(rowKey)) {
> /* Send message to end point */
> MessagingService.getMessagingInstance().sendOneWay(message,
endpoint);
> }
>
> Jun
> IBM Almaden Research Center
> K55/B1, 650 Harry Road, San Jose, CA 95120-6099
>
> junrao@almaden.ibm.com
>
>
> Inactive hide details for Johan Oskarsson ---09/01/2009 12:49:28 PM---I
> have slapped together a basic Hadoop 0.18 CassandraOutpJohan Oskarsson
> ---09/01/2009 12:49:28 PM---I have slapped together a basic Hadoop 0.18
> CassandraOutputFormat based on the code Chris put up.
>
>
> From:
> Johan Oskarsson <jo...@oskarsson.nu>
>
> To:
> cassandra-user@incubator.apache.org
>
> Cc:
> cassandra-dev@incubator.apache.org
>
> Date:
> 09/01/2009 12:49 PM
>
> Subject:
> Re: Cassandra + Hadoop + BMT
>
> ------------------------------------------------------------------------
>
>
>
>
> I have slapped together a basic Hadoop 0.18 CassandraOutputFormat based
> on the code Chris put up.
>
> Usage:
> conf.setOutputKeyClass(RowColumn.class);
> conf.setOutputValueClass(BytesWritable.class);
>
> conf.setOutputFormat(CassandraOutputFormat.class);
> conf.set(CassandraOutputFormat.CONF_COLUMN_FAMILY_NAME,
"columnfamilyname");
> conf.set(CassandraOutputFormat.CONF_KEYSPACE, "keyspacename");
>
> DistributedCache.addCacheFile(new URI("uri_to_storage-conf.xml"), conf);
>
> + your job specific settings.
>
> Then after the job run this method: CassandraOutputFormat.forceFlush
>
> Source code here:
> http://github.com/johanoskarsson/cassandraoutputformat/tree/master
>
> Big thanks to Chris for figuring out the mystery that is BinaryMemtable
>
> /Johan
>
> Chris Goffinet wrote:
>> Hi Guys
>>
>> This is long overdue but I have posted a very rough rough example (with
>> Digg stuff removed) for getting BMT working with Cassandra. Patches are
>> coming next up for the JIRA tickets. I'll try to get a more generic
>> map/reduce job finished by end of the week that integrates Hive output.
>>
>> http://github.com/lenn0x/Cassandra-Hadoop-BMT/tree/master
>>
>> -Chris
>
>
>
Re: Cassandra + Hadoop + BMT
Posted by Johan Oskarsson <jo...@oskarsson.nu>.
In my version of the code the storage endpoints are pulled out from a
seed node using the NodeProbe class and then put into the StorageService
using the updateTokenMetadata method.
See updateTokenMetadata in CassandraClient:
http://github.com/johanoskarsson/cassandraoutputformat/blob/dfa4dbf9b1bc81854b492af14536693002e19e52/src/java/fm/last/hadoop/mapred/CassandraClient.java
Granted it's not a perfect solution.
/Johan
Jun Rao wrote:
> I was trying to understand how the MapReduce job figures out where a row
> is located in a cassandra cluster and I saw the following code. Does
> this really work? To compute the proper endpoints, the StorageService
> needs to be started to obtain all tokens from other nodes through
> gossip. However, StorageService is not started in the MapReduce job.
>
> for (EndPoint endpoint :
> StorageService.instance().getReadStorageEndPoints(rowKey)) {
> /* Send message to end point */
> MessagingService.getMessagingInstance().sendOneWay(message, endpoint);
> }
>
> Jun
> IBM Almaden Research Center
> K55/B1, 650 Harry Road, San Jose, CA 95120-6099
>
> junrao@almaden.ibm.com
>
>
> Inactive hide details for Johan Oskarsson ---09/01/2009 12:49:28 PM---I
> have slapped together a basic Hadoop 0.18 CassandraOutpJohan Oskarsson
> ---09/01/2009 12:49:28 PM---I have slapped together a basic Hadoop 0.18
> CassandraOutputFormat based on the code Chris put up.
>
>
> From:
> Johan Oskarsson <jo...@oskarsson.nu>
>
> To:
> cassandra-user@incubator.apache.org
>
> Cc:
> cassandra-dev@incubator.apache.org
>
> Date:
> 09/01/2009 12:49 PM
>
> Subject:
> Re: Cassandra + Hadoop + BMT
>
> ------------------------------------------------------------------------
>
>
>
>
> I have slapped together a basic Hadoop 0.18 CassandraOutputFormat based
> on the code Chris put up.
>
> Usage:
> conf.setOutputKeyClass(RowColumn.class);
> conf.setOutputValueClass(BytesWritable.class);
>
> conf.setOutputFormat(CassandraOutputFormat.class);
> conf.set(CassandraOutputFormat.CONF_COLUMN_FAMILY_NAME, "columnfamilyname");
> conf.set(CassandraOutputFormat.CONF_KEYSPACE, "keyspacename");
>
> DistributedCache.addCacheFile(new URI("uri_to_storage-conf.xml"), conf);
>
> + your job specific settings.
>
> Then after the job run this method: CassandraOutputFormat.forceFlush
>
> Source code here:
> http://github.com/johanoskarsson/cassandraoutputformat/tree/master
>
> Big thanks to Chris for figuring out the mystery that is BinaryMemtable
>
> /Johan
>
> Chris Goffinet wrote:
>> Hi Guys
>>
>> This is long overdue but I have posted a very rough rough example (with
>> Digg stuff removed) for getting BMT working with Cassandra. Patches are
>> coming next up for the JIRA tickets. I'll try to get a more generic
>> map/reduce job finished by end of the week that integrates Hive output.
>>
>> http://github.com/lenn0x/Cassandra-Hadoop-BMT/tree/master
>>
>> -Chris
>
>
>
Re: Cassandra + Hadoop + BMT
Posted by Jun Rao <ju...@almaden.ibm.com>.
I was trying to understand how the MapReduce job figures out where a row is
located in a cassandra cluster and I saw the following code. Does this
really work? To compute the proper endpoints, the StorageService needs to
be started to obtain all tokens from other nodes through gossip. However,
StorageService is not started in the MapReduce job.
for (EndPoint endpoint : StorageService.instance
().getReadStorageEndPoints(rowKey)) {
/* Send message to end point */
MessagingService.getMessagingInstance().sendOneWay(message,
endpoint);
}
Jun
IBM Almaden Research Center
K55/B1, 650 Harry Road, San Jose, CA 95120-6099
junrao@almaden.ibm.com
|------------>
| From: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|Johan Oskarsson <jo...@oskarsson.nu> |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|------------>
| To: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|cassandra-user@incubator.apache.org |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|------------>
| Cc: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|cassandra-dev@incubator.apache.org |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|------------>
| Date: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|09/01/2009 12:49 PM |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|------------>
| Subject: |
|------------>
>--------------------------------------------------------------------------------------------------------------------------------------------------|
|Re: Cassandra + Hadoop + BMT |
>--------------------------------------------------------------------------------------------------------------------------------------------------|
I have slapped together a basic Hadoop 0.18 CassandraOutputFormat based
on the code Chris put up.
Usage:
conf.setOutputKeyClass(RowColumn.class);
conf.setOutputValueClass(BytesWritable.class);
conf.setOutputFormat(CassandraOutputFormat.class);
conf.set(CassandraOutputFormat.CONF_COLUMN_FAMILY_NAME,
"columnfamilyname");
conf.set(CassandraOutputFormat.CONF_KEYSPACE, "keyspacename");
DistributedCache.addCacheFile(new URI("uri_to_storage-conf.xml"), conf);
+ your job specific settings.
Then after the job run this method: CassandraOutputFormat.forceFlush
Source code here:
http://github.com/johanoskarsson/cassandraoutputformat/tree/master
Big thanks to Chris for figuring out the mystery that is BinaryMemtable
/Johan
Chris Goffinet wrote:
> Hi Guys
>
> This is long overdue but I have posted a very rough rough example (with
> Digg stuff removed) for getting BMT working with Cassandra. Patches are
> coming next up for the JIRA tickets. I'll try to get a more generic
> map/reduce job finished by end of the week that integrates Hive output.
>
> http://github.com/lenn0x/Cassandra-Hadoop-BMT/tree/master
>
> -Chris