cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Agrawal, Pratik" <paagr...@amazon.com.INVALID>
Subject Re: Hinted Handoff
Date Tue, 07 Aug 2018 17:37:51 GMT
Please find my comments inline.

From: kurt greaves <kurt@instaclustr.com>
Reply-To: "user@cassandra.apache.org" <user@cassandra.apache.org>
Date: Tuesday, August 7, 2018 at 1:20 AM
To: User <user@cassandra.apache.org>
Subject: Re: Hinted Handoff

Does Cassandra TTL out the hints after max_hint_window_in_ms? From my understanding, Cassandra
only stops collecting hints after max_hint_window_in_ms but can still keep replaying the hints
if the node comes back again. Is this correct? Is there a way to TTL out hints?

No, but it won't send hints that have passed HH window. Also, this shouldn't be caused by
HH as the hints maintain the original timestamp with which they were written.

  *   We actually saw data resurrecting after HH window. One interesting thing to notice is
that, the data was resurrecting in intervals (after ~1Hr).
  *   Original timestamp doesn’t help since the other copies of the data are actually deleted
and tombstones are wiped out after 15 minutes.
  *   The Cassandra version we are using is 2.2.8

Honestly, this sounds more like a use case for a distributed cache rather than Cassandra.
Keeping data for 30 minutes and then deleting it is going to be a nightmare to manage in Cassandra.

  *   Agreed, we are looking into other databases (like Redis, Aerospike). We have a write
heavy use case and also need optimistic locking + columnar updates.

Thanks,
Pratik

On 7 August 2018 at 07:20, Agrawal, Pratik <paagrawa@amazon.com.invalid<mailto:paagrawa@amazon.com.invalid>>
wrote:
Does Cassandra TTL out the hints after max_hint_window_in_ms? From my understanding, Cassandra
only stops collecting hints after max_hint_window_in_ms but can still keep replaying the hints
if the node comes back again. Is this correct? Is there a way to TTL out hints?

Thanks,
Pratik

From: Kyrylo Lebediev <Kyrylo_Lebediev@epam.com<mailto:Kyrylo_Lebediev@epam.com>.INVALID>
Reply-To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" <user@cassandra.apache.org<mailto:user@cassandra.apache.org>>
Date: Monday, August 6, 2018 at 4:10 PM
To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" <user@cassandra.apache.org<mailto:user@cassandra.apache.org>>
Subject: Re: Hinted Handoff


Small gc_grace_seconds value lowers max allowed node downtime, which is 15 minutes in your
case. After 15 minutes of downtime you'll need to replace the node, as you described. This
interval looks too short to be able to do planned maintenance. So, in case you set larger
value for gc_grace_seconds (lets say, hours or a day) will you get visible read amplification
/ waste a lot of disk space / issues with compactions?



Hinted handoff may be the reason in case hinted handoff window is longer than gc_grace_seconds.
To me it looks like hinted handoff window (max_hint_window_in_ms in cassandra.yaml, which
defaults to 3h) must always be set to a value less than gc_grace_seconds.



Regards,

Kyrill

________________________________
From: Agrawal, Pratik <paagrawa@amazon.com.INVALID>
Sent: Monday, August 6, 2018 8:22:27 PM
To: user@cassandra.apache.org<mailto:user@cassandra.apache.org>
Subject: Hinted Handoff


Hello all,

We use Cassandra in non-conventional way, where our data is short termed (life cycle of about
20-30 minutes) where each record is updated ~5 times and then deleted. We have GC grace of
15 minutes.

We are seeing 2 problems

1.) A certain number of Cassandra nodes goes down and then we remove it from the cluster using
Cassandra removenode command and replace the dead nodes with new nodes. While new nodes are
joining in, we see more nodes down (which are not actually down) but we see following errors
in the log

“Gossip not settled after 321 polls. Gossip Stage active/pending/completed: 1/816/0”



To fix the issue, I restarted the server and the nodes now appear to be up and the problem
is solved



Can this problem be related to https://issues.apache.org/jira/browse/CASSANDRA-6590 ?



2.) Meanwhile, after restarting the nodes mentioned above, we see that some old deleted data
is resurrected (because of short lifecycle of our data). My guess at the moment is that these
data is resurrected due to hinted handoff. Interesting point to note here is that data keeps
resurrecting at periodic intervals (like an hour) and then finally stops. Could this be caused
by hinted handoff? if so is there any setting which we can set to specify that “invalidate,
hinted handoff data after 5-10 minutes”.



Thanks,
Pratik

Mime
View raw message