Turn the logging up to DEBUG, if you want to you can do it just for org.apache.cassandra.service.StorageProxy , and look for…
* forwarding counter update of key… when the coordinator is forwarding the counter update * insert writing local & replicate… when a counter mutation is applied locally * insert writing key… when the ROW is been sent btw I was wrong about the sort by proximity. The default SimpleSnitch does no sorting, so the endpoint order is the token order. The DynamicSnitch will re-order according to the observed latency. Hope that helps. Aaron ----------------- Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 13 Aug 2011, at 12:52, Ian Danforth wrote: > Thanks! This would make sense but I'm not sure it maps to what I'm seeing. > Specifically if I start processes that write to each of the three nodes, so I > know there is a constant stream of writes to each node, instead of seeing > replicate on write on all nodes, I see it only on a single node. > > It appears that all increments are being routed to a single node which then > has to do all replication. However I don't have a good way to see the write > flow > > Connection made to node: XXX > Write sent to node: XXX > Replicas sent to nodes: XXX, XXX > > If I could see that I'd have a lot more confidence about what's going on. > > Ian > > On Thu, Aug 11, 2011 at 6:38 PM, aaron morton <aa...@thelastpickle.com> wrote: > I've not checked the code but from memory when the nodes are ordered in > proximity to the coordinator the local node is always first if it's in the > replica set. So with RF=3 and N=3 the closest node is always the local one. > > Cheers > > ----------------- > Aaron Morton > Freelance Cassandra Developer > @aaronmorton > http://www.thelastpickle.com > > On 12 Aug 2011, at 10:10, Ian Danforth wrote: > >> I'm writing at QUORUM though and (pardon me for being dense) it would seem >> that the first replica shouldn't always be on the same server if I'm using >> RP. >> >> I very much appreciate your time, I'm sure there is something basic that >> just hasn't clicked. >> >> Ian >> >> On Thu, Aug 11, 2011 at 2:56 PM, Jonathan Ellis <jbel...@gmail.com> wrote: >> https://issues.apache.org/jira/browse/CASSANDRA-2889 >> >> On Thu, Aug 11, 2011 at 2:55 PM, Ian Danforth <idanfo...@numenta.com> wrote: >> > I don't think so, perhaps more succinctly, why would ROW actions only be >> > performed on a single node? >> > Ian >> > >> > On Wed, Aug 10, 2011 at 8:12 PM, Jonathan Ellis <jbel...@gmail.com> wrote: >> >> >> >> are you i/o bound? I believe counter ROW needs to perform a read of >> >> the old value. >> >> >> >> On Wed, Aug 10, 2011 at 7:17 PM, Ian Danforth <idanfo...@numenta.com> >> >> wrote: >> >> > Hello all! >> >> > Thanks for taking the time to read this, I'm new to Cassandra and really >> >> > want to get it working :) >> >> > Below you'll find the output from tpstats on the three nodes in my .8.1 >> >> > cluster. As you'll note from ring (also below), my cluster appears >> >> > balanced. >> >> > However, please note that NODE1 is the only node that has completed any >> >> > ReplicateOnWriteStage actions. I've been having a lot of trouble with >> >> > RoW >> >> > and seen a single node in my various test clusters get hugely backed up >> >> > on >> >> > this stage to the point where it slows all writes to a crawl. >> >> > I'm using counters in a column family defined like this: >> >> > ==== KEYSPACE/CF ==== >> >> > create keyspace Keyspace1 >> >> > with strategy_options=[{replication_factor:3}] >> >> > and placement_strategy = >> >> > 'org.apache.cassandra.locator.SimpleStrategy'; >> >> > use Keyspace1; >> >> > create column family TwitterTest >> >> > with default_validation_class = CounterColumnType >> >> > and replicate_on_write = true; >> >> > ==== >> >> > >> >> > With any given batch_insert() I'm incrementing ~17,000 counters. I've >> >> > backed >> >> > off a bit and started using ConsistencyLevel.QUORUM, as it was suggested >> >> > elsewhere that ANY or ONE could overwhelm the system. >> >> > >> >> > I hope I'm just doing something stupid and one you experienced folks can >> >> > point out my error. >> >> > Thanks in advance! >> >> > Ian >> >> > ==== MORE DATA ==== >> >> > Address DC Rack Status State Load >> >> > Owns >> >> > Token >> >> > >> >> > 113427455640312821154458202477256070484 >> >> > 10.87.9.57 datacenter1 rack1 Up Normal 193.86 MB >> >> > 33.33% 0 >> >> > 10.86.223.32 datacenter1 rack1 Up Normal 191.1 MB >> >> > 33.33% 56713727820156410577229101238628035242 >> >> > 10.86.229.24 datacenter1 rack1 Up Normal 193.63 MB >> >> > 33.33% 113427455640312821154458202477256070484 >> >> > >> >> > >> >> > >> >> > SEED >> >> > Pool Name Active Pending Completed >> >> > ReadStage 0 0 7 >> >> > RequestResponseStage 0 0 1479839 >> >> > MutationStage 0 0 1479908 >> >> > ReadRepairStage 0 0 0 >> >> > ReplicateOnWriteStage 0 0 0 >> >> > GossipStage 0 0 5176 >> >> > AntiEntropyStage 0 0 0 >> >> > MigrationStage 0 0 128 >> >> > MemtablePostFlusher 0 0 28 >> >> > StreamStage 0 0 0 >> >> > FlushWriter 0 0 28 >> >> > MiscStage 0 0 2 >> >> > FlushSorter 0 0 0 >> >> > InternalResponseStage 0 0 363 >> >> > HintedHandoff 0 0 0 >> >> > NODE1 >> >> > Pool Name Active Pending Completed >> >> > ReadStage 0 0 7 >> >> > RequestResponseStage 0 0 2960208 >> >> > MutationStage 0 0 1480104 >> >> > ReadRepairStage 0 0 0 >> >> > ReplicateOnWriteStage 0 0 1480104 >> >> > GossipStage 0 0 4151 >> >> > AntiEntropyStage 0 0 0 >> >> > MigrationStage 0 0 128 >> >> > MemtablePostFlusher 0 0 31 >> >> > StreamStage 0 0 8 >> >> > FlushWriter 0 0 31 >> >> > MiscStage 0 0 8 >> >> > FlushSorter 0 0 0 >> >> > InternalResponseStage 0 0 1 >> >> > HintedHandoff 0 0 0 >> >> > NODE2 >> >> > Pool Name Active Pending Completed >> >> > ReadStage 0 0 7 >> >> > RequestResponseStage 0 0 0 >> >> > MutationStage 0 0 1481530 >> >> > ReadRepairStage 0 0 0 >> >> > ReplicateOnWriteStage 0 0 0 >> >> > GossipStage 0 0 4201 >> >> > AntiEntropyStage 0 0 0 >> >> > MigrationStage 0 0 128 >> >> > MemtablePostFlusher 0 0 31 >> >> > StreamStage 0 0 6 >> >> > FlushWriter 0 0 31 >> >> > MiscStage 0 0 6 >> >> > FlushSorter 0 0 0 >> >> > InternalResponseStage 0 0 1 >> >> > HintedHandoff 0 0 0 >> >> >> >> >> >> >> >> -- >> >> Jonathan Ellis >> >> Project Chair, Apache Cassandra >> >> co-founder of DataStax, the source for professional Cassandra support >> >> http://www.datastax.com >> > >> > >> >> >> >> -- >> Jonathan Ellis >> Project Chair, Apache Cassandra >> co-founder of DataStax, the source for professional Cassandra support >> http://www.datastax.com >> > >