You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "debadatta das (JIRA)" <ji...@apache.org> on 2012/10/11 17:25:02 UTC

[jira] [Updated] (CASSANDRA-4794) cassandra 1.2.0 beta: atomic_batch_mutate fails with Default TException

     [ https://issues.apache.org/jira/browse/CASSANDRA-4794?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

debadatta das updated CASSANDRA-4794:
-------------------------------------

    Description: 
Hi,
We have installed cassandra 1.2.0 beta with thrift 0.7.0. We are using cpp interface. When we use batch_mutate API, it works fine. But when we are using the new atomic_batch_mutate API with same parameters as batch_mutate, it fails with org::apache::cassandra::TimedOutException, what(): Default TException. We get the same TException error even after increasing Send/Reciv timeout values of Tsocket to 15 seconds or more.

Details:
cassandra ring:
cassandra ring with single node
consistency level paramter to atomic_batch_mutate
ConsistencyLevel::ONE
Thrift version:
same results with thrift 0.5.0 and thrift 0.7.0.
thrift 0.8.0 seems unsupported with cassanda 1.2.0. Gives compilation error for cpp interface build.

We are calling atomic_batch_mutate() with same parameters as batch_mutate.
cassclient.atomic_batch_mutate(outermap1, ConsistencyLevel::ONE);

where outmap1 is
map<string, map<string, vector<Mutation> > > outermap1;

Please point out if anything is missing while using atomic_batch_mutate or the reason behind the failure.

The logs in cassandra system.log we get during atomic_batch_mutate failure are:

INFO [ScheduledTasks:1] 2012-10-10 04:47:30,604 MessagingService.java (line 800) 1 MUTATION messages dropped in last 5000ms
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,606 StatusLogger.java (line 53) Pool Name Active Pending Blocked
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,607 StatusLogger.java (line 68) ReadStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) RequestResponseStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) ReadRepairStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) MutationStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) ReplicateOnWriteStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) GossipStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) AntiEntropyStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) MigrationStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) StreamStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) MemtablePostFlusher 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) FlushWriter 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) MiscStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) commitlog_archiver 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) InternalResponseStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 73) CompactionManager 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 85) MessagingService n/a 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 95) Cache Type Size Capacity KeysToSave Provider
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 96) KeyCache 227 74448896 all
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 102) RowCache 0 0 all org.apache.cassandra.cache.SerializingCacheProvider
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 109) ColumnFamily Memtable ops,data
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) KeyspaceTest.CF_Test 1,71
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.local 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.peers 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.batchlog 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.NodeIdInfo 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.LocationInfo 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.Schema 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.Migrations 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_keyspaces 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_columns 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_columnfamilies 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.IndexInfo 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.range_xfers 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.hints 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.HintsColumnFamily 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,614 StatusLogger.java (line 112) system_traces.sessions 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,614 StatusLogger.java (line 112) system_traces.events 0,0
ERROR [EXPIRING-MAP-REAPER:1] 2012-10-10 04:47:41,857 CassandraDaemon.java (line 132) Exception in thread Thread[EXPIRING-MAP-REAPER:1,5,main]
java.lang.AssertionError: rwcvmx84q0379/10.17.122.99
at org.apache.cassandra.service.StorageProxy.scheduleLocalHint(StorageProxy.java:565)
at org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:333)
at org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:321)
at org.apache.cassandra.utils.ExpiringMap$1.run(ExpiringMap.java:94)
at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:75)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:679) 


A sample program in CPP we have tested is attached.

Regards,
Debadatta,
Openwave 


  was:
Hi,
We have installed cassandra 1.2.0 beta with thrift 0.7.0. We are using cpp interface. When we use batch_mutate API, it works fine. But when we are using the new atomic_batch_mutate API with same parameters as batch_mutate, it fails with org::apache::cassandra::TimedOutException, what(): Default TException. We get the same TException error even after increasing Send/Reciv timeout values of Tsocket to 15 seconds or more.

Details:
cassandra ring:
cassandra ring with single node
consistency level paramter to atomic_batch_mutate
ConsistencyLevel::ONE
Thrift version:
same results with thrift 0.5.0 and thrift 0.7.0.
thrift 0.8.0 seems unsupported with cassanda 1.2.0. Gives compilation error for cpp interface build.

We are calling atomic_batch_mutate() with same parameters as batch_mutate.
cassclient.atomic_batch_mutate(outermap1, ConsistencyLevel::ONE);

where outmap1 is
map<string, map<string, vector<Mutation> > > outermap1;

Please point out if anything is missing while using atomic_batch_mutate or the reason behind the failure.

The logs in cassandra system.log we get during atomic_batch_mutate failure are:

INFO [ScheduledTasks:1] 2012-10-10 04:47:30,604 MessagingService.java (line 800) 1 MUTATION messages dropped in last 5000ms
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,606 StatusLogger.java (line 53) Pool Name Active Pending Blocked
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,607 StatusLogger.java (line 68) ReadStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) RequestResponseStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) ReadRepairStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) MutationStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) ReplicateOnWriteStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) GossipStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) AntiEntropyStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) MigrationStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) StreamStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) MemtablePostFlusher 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) FlushWriter 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) MiscStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) commitlog_archiver 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) InternalResponseStage 0 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 73) CompactionManager 0 0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 85) MessagingService n/a 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 95) Cache Type Size Capacity KeysToSave Provider
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 96) KeyCache 227 74448896 all
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 102) RowCache 0 0 all org.apache.cassandra.cache.SerializingCacheProvider
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 109) ColumnFamily Memtable ops,data
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) KeyspaceTest.CF_Test 1,71
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.local 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.peers 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.batchlog 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.NodeIdInfo 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.LocationInfo 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.Schema 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.Migrations 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_keyspaces 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_columns 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_columnfamilies 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.IndexInfo 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.range_xfers 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.hints 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.HintsColumnFamily 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,614 StatusLogger.java (line 112) system_traces.sessions 0,0
INFO [ScheduledTasks:1] 2012-10-10 04:47:30,614 StatusLogger.java (line 112) system_traces.events 0,0
ERROR [EXPIRING-MAP-REAPER:1] 2012-10-10 04:47:41,857 CassandraDaemon.java (line 132) Exception in thread Thread[EXPIRING-MAP-REAPER:1,5,main]
java.lang.AssertionError: rwcvmx84q0379/10.17.122.99
at org.apache.cassandra.service.StorageProxy.scheduleLocalHint(StorageProxy.java:565)
at org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:333)
at org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:321)
at org.apache.cassandra.utils.ExpiringMap$1.run(ExpiringMap.java:94)
at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:75)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:679) 


A sample program in CPP we have tested with is:

try{
boost::shared_ptr<TSocket> socket = boost::shared_ptr<TSocket>(new TSocket("10.17.122.99", 9160));
boost::shared_ptr<TTransport> tr = boost::shared_ptr<TFramedTransport>(new TFramedTransport (socket));
boost::shared_ptr<TProtocol> p = boost::shared_ptr<TBinaryProtocol>(new TBinaryProtocol(tr));
CassandraClient cass(p);
tr->open();
cass.set_keyspace("KeyspaceTest");

vector<Column> columnvector1;
Column c1;
c1.name="ver";
c1.value="1";
c1.__isset.value = true;
setTimestamp(c1);

Column c2;
c2.name="flags";
c2.value="000";
c2.__isset.value = true;
setTimestamp(c2);

Column c3;
c3.name="adminstate";
c3.value="U";
c3.__isset.value = true;
setTimestamp(c3);

Column c4;
c4.name="Deleted";
c4.value="0";
c4.__isset.value = true;
setTimestamp(c4);

columnvector1.push_back(c1);
columnvector1.push_back(c2);
columnvector1.push_back(c3);
columnvector1.push_back(c4);

SuperColumn sc1;
sc1.name = "info";
sc1.columns = columnvector1;

ColumnOrSuperColumn csc1;
csc1.__isset.super_column = true;
csc1.super_column = sc1;

Mutation mutation1;
mutation1.__isset.column_or_supercolumn = true;
mutation1.column_or_supercolumn = csc1;

vector<Mutation> mutationlist1;
mutationlist1.push_back(mutation1);

map<string, vector<Mutation> > mutationmap1;
mutationmap1["CF_Test"] = mutationlist1;

string key = "2012";
map<string, map<string, vector<Mutation> > > outermap1;
outermap1[key] = mutationmap1;

cass.atomic_batch_mutate(outermap1, ConsistencyLevel::ONE);

tr->close();

}catch(TTransportException te){
printf("Exception: %s [%d]\n", te.what(), te.getType());
}catch(InvalidRequestException ire){
printf("Exception: %s [%s]\n", ire.what(), ire.why.c_str());
}catch(NotFoundException nfe){
printf("Exception: %s\n", nfe.what());
}catch(TimedOutException toe){
printf("Exception: %s\n", toe.what());
}

return 0; 



Regards,
Debadatta,
Openwave 


    
> cassandra 1.2.0 beta: atomic_batch_mutate fails with Default TException
> -----------------------------------------------------------------------
>
>                 Key: CASSANDRA-4794
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4794
>             Project: Cassandra
>          Issue Type: Bug
>          Components: API
>    Affects Versions: 1.2.0 beta 1
>         Environment: C++
>            Reporter: debadatta das
>
> Hi,
> We have installed cassandra 1.2.0 beta with thrift 0.7.0. We are using cpp interface. When we use batch_mutate API, it works fine. But when we are using the new atomic_batch_mutate API with same parameters as batch_mutate, it fails with org::apache::cassandra::TimedOutException, what(): Default TException. We get the same TException error even after increasing Send/Reciv timeout values of Tsocket to 15 seconds or more.
> Details:
> cassandra ring:
> cassandra ring with single node
> consistency level paramter to atomic_batch_mutate
> ConsistencyLevel::ONE
> Thrift version:
> same results with thrift 0.5.0 and thrift 0.7.0.
> thrift 0.8.0 seems unsupported with cassanda 1.2.0. Gives compilation error for cpp interface build.
> We are calling atomic_batch_mutate() with same parameters as batch_mutate.
> cassclient.atomic_batch_mutate(outermap1, ConsistencyLevel::ONE);
> where outmap1 is
> map<string, map<string, vector<Mutation> > > outermap1;
> Please point out if anything is missing while using atomic_batch_mutate or the reason behind the failure.
> The logs in cassandra system.log we get during atomic_batch_mutate failure are:
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,604 MessagingService.java (line 800) 1 MUTATION messages dropped in last 5000ms
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,606 StatusLogger.java (line 53) Pool Name Active Pending Blocked
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,607 StatusLogger.java (line 68) ReadStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) RequestResponseStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) ReadRepairStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) MutationStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,608 StatusLogger.java (line 68) ReplicateOnWriteStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) GossipStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) AntiEntropyStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) MigrationStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) StreamStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,609 StatusLogger.java (line 68) MemtablePostFlusher 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) FlushWriter 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) MiscStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) commitlog_archiver 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 68) InternalResponseStage 0 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,610 StatusLogger.java (line 73) CompactionManager 0 0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 85) MessagingService n/a 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 95) Cache Type Size Capacity KeysToSave Provider
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 96) KeyCache 227 74448896 all
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,611 StatusLogger.java (line 102) RowCache 0 0 all org.apache.cassandra.cache.SerializingCacheProvider
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 109) ColumnFamily Memtable ops,data
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) KeyspaceTest.CF_Test 1,71
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.local 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.peers 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.batchlog 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.NodeIdInfo 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.LocationInfo 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,612 StatusLogger.java (line 112) system.Schema 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.Migrations 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_keyspaces 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_columns 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.schema_columnfamilies 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.IndexInfo 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.range_xfers 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.hints 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,613 StatusLogger.java (line 112) system.HintsColumnFamily 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,614 StatusLogger.java (line 112) system_traces.sessions 0,0
> INFO [ScheduledTasks:1] 2012-10-10 04:47:30,614 StatusLogger.java (line 112) system_traces.events 0,0
> ERROR [EXPIRING-MAP-REAPER:1] 2012-10-10 04:47:41,857 CassandraDaemon.java (line 132) Exception in thread Thread[EXPIRING-MAP-REAPER:1,5,main]
> java.lang.AssertionError: rwcvmx84q0379/10.17.122.99
> at org.apache.cassandra.service.StorageProxy.scheduleLocalHint(StorageProxy.java:565)
> at org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:333)
> at org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:321)
> at org.apache.cassandra.utils.ExpiringMap$1.run(ExpiringMap.java:94)
> at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:75)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
> at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
> at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:679) 
> A sample program in CPP we have tested is attached.
> Regards,
> Debadatta,
> Openwave 

--
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