Just noticed this on your email 

> ps: all nodes are cassandra-1.1.6-dse-p1

If you are on DSE I'd recommend hitting their forums or the support channels.

Cheers
 
-----------------
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 26/03/2013, at 1:41 AM, Andras Szerdahelyi 
<andras.szerdahe...@ignitionone.com> wrote:

> Thanks again!
> 
> Nodetool gossipinfo correctly lists the existing nodes only
> 
> - last change to ring topology was months ago
> - I started the problem node with -Dcassanda.load_ring_state=false and
> observed no unusual behaviour ( this is with hinted handoff OFF. With
> hinted handoff ON I see the same behaviour as before )
> - tried to assassinate these endpoints but got UnknownHostExceptions for
> all 3
> - tried remove token but got java.lang.UnsupportedOperationException:
> Token not found. For all of these
> 
> I have an update however. I changed which node I use to coordinate
> mutations and its happening elsewhere too, same tokens.
> I'm clueless as to what could have caused my ring to end up in such an
> inconsistent state.. How can there be pending hints for endpoints
> gossipinfo does not know about?
> 
> Regards,
> Andras
> 
> 
> On 21/03/13 17:56, "aaron morton" <aa...@thelastpickle.com> wrote:
> 
>> Take a look a nodetool gossipinfo it will tell you what nodes the node
>> thinks are around.
>> 
>> If you can see something in gossip that should not be there you have a
>> few choices.
>> 
>> * if it's less than 3 days since a change to ring topology wait and see
>> if C* sorts it out.
>> * try restarting nodes with -Dcassanda.load_ring_state=false as a JVM opt
>> in cassandra-env.sh. This may not work because when the node restarts
>> others will tell it the bad info
>> * try the unsafeAssassinateEndpoint() call on the Gossiper MBean via JMX
>> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassand
>> ra/gms/GossiperMBean.java#L28
>> 
>> Cheers
>> 
>> -----------------
>> Aaron Morton
>> Freelance Cassandra Consultant
>> New Zealand
>> 
>> @aaronmorton
>> http://www.thelastpickle.com
>> 
>> On 20/03/2013, at 11:10 PM, Andras Szerdahelyi
>> <andras.szerdahe...@ignitionone.com> wrote:
>> 
>>> Thanks, Aaron.
>>> 
>>> I re-enabled hinted handoff and noted the following
>>>     € no host is marked down in nodetool ring
>>>     € No host is logged as down or dead in logs
>>>     € No "started hinted handoff for.." is logged
>>>     € The hinted handoff manager Mbean lists pending hints to ..
>>> (drumroll) 3 non-existent nodes?
>>> 
>>> Here's my ring
>>> 
>>> Note: Ownership information does not include topology, please specify a
>>> keyspace. 
>>> Address         DC          Rack        Status State   Load
>>> Owns                Token
>>> 
>>>                  113427455640312821154458202477256070785
>>> XX.XX.1.113    ione-us-atl rack1       Up     Normal  382.08 GB
>>> 33.33%              0
>>> XX.XX.31.10      ione-us-lvg rack1       Up     Normal  266.04 GB
>>> 0.00%               100
>>> XX.XX.0.71     ione-be-bru rack1       Up     Normal  85.86 GB
>>> 0.00%               200
>>> XX.XX.2.86     ione-analytics-us-atlrack1       Up     Normal  153.6 GB
>>>      0.00%               300
>>> XX.XX.1.45     ione-us-atl-ssdrack1       Up     Normal  296.72 GB
>>> 0.00%               400
>>> XX.XX.2.85     ione-analytics-us-atlrack1       Up     Normal  100.3 GB
>>>      33.33%              56713727820156410577229101238628035542
>>> XX.XX.1.204    ione-us-atl rack1       Up     Normal  341.55 GB
>>> 16.67%              85070591730234615865843651857942052864
>>> XX.XX.11      ione-us-lvg rack1       Up     Normal  320.22 GB
>>> 0.00%               85070591730234615865843651857942052964
>>> XX.XX.2.87     ione-analytics-us-atlrack1       Up     Normal  166.48
>>> GB       16.67%              113427455640312821154458202477256070785
>>> 
>>> And these are nodes pending hints according to the Mbean
>>> 
>>> 166860289390734216023086131251507064403
>>> 143927757573010354572009627285182898319
>>> 24295500190543334543807902779534181373
>>> 
>>> Err.. Unbalanced ring ? Opscenter says otherwise ( "OpsCenter has
>>> detected that the token ranges are evenly distributed across the nodes
>>> in each data center. Load rebalancing is not necessary at this time." )
>>> 
>>> I appreciate your help so far! In the mean time hintedhandOFF because
>>> my mutation TP can't keep up with this traffic, not to mention
>>> compaction..
>>> 
>>> Thanks,
>>> Andras
>>> 
>>> ps: all nodes are cassandra-1.1.6-dse-p1
>>> 
>>> From: aaron morton <aa...@thelastpickle.com>
>>> Reply-To: "user@cassandra.apache.org" <user@cassandra.apache.org>
>>> Date: Monday 18 March 2013 17:51
>>> To: "user@cassandra.apache.org" <user@cassandra.apache.org>
>>> Subject: Re: 33million hinted handoffs from nowhere
>>> 
>>> You can check which nodes hints are being held for using the JMX api.
>>> Look for the org.apache.cassandra.db:type=HintedHandoffManager MBean and
>>> call the listEndpointsPendingHints() function.
>>> 
>>> There are two points where hints may be stored, if the node is down
>>> when the request started or if the node timed out and did not return
>>> before rpc_timeout. To check for the first, look for log lines about a
>>> node being "dead" on the coordinator. To check for the second look for
>>> dropped messages on the other nodes. This will be logged, or you can use
>>> nodetool tpstats to look for them.
>>> 
>>> Cheers
>>> 
>>> -----------------
>>> Aaron Morton
>>> Freelance Cassandra Consultant
>>> New Zealand
>>> 
>>> @aaronmorton
>>> http://www.thelastpickle.com
>>> 
>>> On 15/03/2013, at 2:30 AM, Andras Szerdahelyi
>>> <andras.szerdahe...@ignitionone.com> wrote:
>>> 
>>>> ( The previous letter was sent prematurely, sorry. )
>>>> 
>>>> This node is the only node being written to, but the Cfs being written
>>>> replicate to almost all of the other nodes
>>>> My understanding is that hinted handoff is mutations kept around on
>>>> the coordinator node, to be replayed when the target node re-appears on
>>>> the ring. All my nodes are up and again, no hinted handoff is logged on
>>>> the node itself
>>>> 
>>>> Thanks!
>>>> Andras
>>>> 
>>>> From: Andras Szerdahelyi <andras.szerdahe...@ignitionone.com>
>>>> Date: Thursday 14 March 2013 14:25
>>>> To: "user@cassandra.apache.org" <user@cassandra.apache.org>
>>>> Subject: 33million hinted handoffs from nowhere
>>>> 
>>>> Hi list,
>>>> 
>>>> I am experiencing seemingly uncontrollable and unexplained growth of
>>>> my HintedHandoff CF on a single node. Unexplained because there are no
>>>> hinted handoffs being logged on the node, uncontrollable because I see
>>>> 33 million inserts in cfstats and the size of the stables is over 10
>>>> gigs all in an hour of uptime.
>>>> 
>>>> 
>>>> I have done the following to try and reproduce this:
>>>> 
>>>> - shut down my cluster
>>>> - on all nodes: remove sstables from the HintsColumnFamily data dir
>>>> - on all nodes: remove commit logs
>>>> - start all nodes but the one that¹s showing this problem
>>>> - nothing is writing to any of the nodes. There are no hinted handoff
>>>> going on anywhere
>>>> - bring back the node in question last
>>>> - few seconds after boot:
>>>> 
>>>>                Column Family: HintsColumnFamily
>>>>                SSTable count: 1
>>>>                Space used (live): 44946532
>>>>                Space used (total): 44946532
>>>>                Number of Keys (estimate): 256
>>>>                Memtable Columns Count: 17840
>>>>                Memtable Data Size: 17569909
>>>>                Memtable Switch Count: 2
>>>>                Read Count: 0
>>>>                Read Latency: NaN ms.
>>>>                Write Count: 184836
>>>>                Write Latency: 0.668 ms.
>>>>                Pending Tasks: 0
>>>>                Bloom Filter False Postives: 0
>>>>                Bloom Filter False Ratio: 0.00000
>>>>                Bloom Filter Space Used: 16
>>>>                Compacted row minimum size: 20924301
>>>>                Compacted row maximum size: 25109160
>>>>                Compacted row mean size: 25109160
>>>> 
>>>> 
>>>> 
>>>> 
>>> 
>> 
> 

Reply via email to