Please find my comments inline.
From: kurt greaves
Reply-To: "user@cassandra.apache.org"
Date: Tuesday, August 7, 2018 at 1:20 AM
To: User
Subject: Re: Hinted Handoff
Does Cassandra TTL out the hints after max_hint_window_in_ms? From my
understanding, Cassandra only stops collec
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
> > > Reply-To: "user@cassandra.apac
ll 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
> *Reply-To: *"user@cassandra.apache.org"
> *Date: *Monday, August 6, 2
: Kyrylo Lebediev
Reply-To: "user@cassandra.apache.org"
Date: Monday, August 6, 2018 at 4:10 PM
To: "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 dow
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_secon
Bit more information. Using jmxterm and inspecting the state of a node when
it's "slow" playing hints, I can see the following from the node that has
hints to play:
$>get MaxHintsInProgress
#mbean = org.apache.cassandra.db:type=StorageProxy:
MaxHintsInProgress = 2048;
$>get HintsInProgress
#mbean
Nope. I added millions of records and several GB to the cluster while one node
was down, and then ran "nodetool flush system hints" on a couple of nodes that
were up, and system/hints has less than 200K in it.
Here’s the relevant part of "nodetool cfstats system.hints":
Keyspace: system
Hi Robert ,
Maybe you need to flush your memtables to actually see the disk usage increase?
This applies to both hosts.
Cheers,
Jens
On Sun, Dec 14, 2014 at 3:52 PM, Robert Wille wrote:
> I have a cluster with RF=3. If I shut down one node, add a bunch of data to
> the cluster, I don’t see a
I’ve got "hinted_handoff_enabled: true" in cassandra.yaml. My settings are all
default except for the DC, listen addresses and snitch. I should have mentioned
this in my original post.
On Dec 14, 2014, at 8:02 AM, Rahul Neelakantan wrote:
> http://www.datastax.com/documentation/cassandra/2.0/c
http://www.datastax.com/documentation/cassandra/2.0/cassandra/configuration/configCassandra_yaml_r.html?scroll=reference_ds_qfg_n1r_1k__hinted_handoff_enabled
Rahul
> On Dec 14, 2014, at 9:46 AM, Robert Wille wrote:
>
> I have a cluster with RF=3. If I shut down one node, add a bunch of data to
It contains mutation (data) that is to be sent to proper endpoint.
M.
W dniu 25.03.2013 20:15, Kanwar Sangha pisze:
Hi - Quick question. Do hints contain the actual data or the data is read from
the SStables and then sent to the other node when it comes up ?
Thanks,
Kanwar
o, if you have enough hints being created that its significantly
> impacting your heap I have a feeling things are going to get out of sync
> very quickly.
>
> On Mar 4, 2013, at 9:17 PM, "Wz1975" wrote:
>
> Why do you think disabling hinted handoff will improve memory u
your heap I have a feeling things are going to get out of sync
>> very quickly.
>>
>> On Mar 4, 2013, at 9:17 PM, "Wz1975" wrote:
>>
>> Why do you think disabling hinted handoff will improve memory usage?
>>
>>
>> Thanks.
>> -Wei
&
tly
> impacting your heap I have a feeling things are going to get out of sync
> very quickly.
>
> On Mar 4, 2013, at 9:17 PM, "Wz1975" wrote:
>
> Why do you think disabling hinted handoff will improve memory usage?
>
>
> Thanks.
> -Wei
>
> Sent from
i Aaron - This was true for pre 1.2 but with 1.2 and virtual nodes, does
>> this still hold ? 1 TB at 1Gb/s will take roughly 2.2hrs assume we stream
>> from say 100 nodes...
>>
>>
>> From: aaron morton [mailto:aa...@thelastpickle.com]
>> Sent: 06 March 2013 2
oes this
still hold ? 1 TB at 1Gb/s will take roughly 2.2hrs assume we stream from say
100 nodes...
From: aaron morton [mailto:aa...@thelastpickle.com]
Sent: 06 March 2013 23:47
To: user@cassandra.apache.org
Subject: Re: Hinted handoff
Check the IO utilisation using iostat
You *really* should
tream from say
100 nodes...
From: aaron morton [mailto:aa...@thelastpickle.com]
Sent: 06 March 2013 23:47
To: user@cassandra.apache.org
Subject: Re: Hinted handoff
Check the IO utilisation using iostat
You *really* should not need to make HH run faster, if you do there is some
thing bad going
:kan...@mavenir.com]
> Sent: 06 March 2013 15:11
> To: user@cassandra.apache.org
> Subject: RE: Hinted handoff
>
> After trying to bump up the “hinted_handoff_throttle_in_kb” to 1G/b per sec,
> It still does not go above 25Mb/s. Is there a limitation ?
>
>
>
&g
2013 15:11
To: user@cassandra.apache.org
Subject: RE: Hinted handoff
After trying to bump up the "hinted_handoff_throttle_in_kb" to 1G/b per sec, It
still does not go above 25Mb/s. Is there a limitation ?
From: Kanwar Sangha [mailto:kan...@mavenir.com]
Sent: 06 March 2013 14:4
After trying to bump up the "hinted_handoff_throttle_in_kb" to 1G/b per sec, It
still does not go above 25Mb/s. Is there a limitation ?
From: Kanwar Sangha [mailto:kan...@mavenir.com]
Sent: 06 March 2013 14:41
To: user@cassandra.apache.org
Subject: RE: Hinted handoff
Got the par
Got the param. thanks
From: Kanwar Sangha [mailto:kan...@mavenir.com]
Sent: 06 March 2013 13:50
To: user@cassandra.apache.org
Subject: Hinted handoff
Hi - Is there a way to increase the hinted handoff throughput ? I am seeing
around 8Mb/s (bits).
Thanks,
Kanwar
p I have a feeling things are going to get out of sync very quickly.
>
> On Mar 4, 2013, at 9:17 PM, "Wz1975" wrote:
>
>> Why do you think disabling hinted handoff will improve memory usage?
>>
>>
>> Thanks.
>> -Wei
>>
>> Sent from my
ove memory usage?
Thanks.
-Wei
Sent from my Samsung smartphone on AT&T
Original message ----
Subject: Re: hinted handoff disabling trade-offs
From: Michael Kjellman mailto:mkjell...@barracuda.com>>
To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org&g
Why do you think disabling hinted handoff will improve memory usage?
Thanks.
-Wei
Sent from my Samsung smartphone on AT&T
Original message
Subject: Re: hinted handoff disabling trade-offs
From: Michael Kjellman
To: "user@cassandra.apache.org"
CC:
Repai
Repair is slow.
On Mar 4, 2013, at 8:07 PM, "Matt Kap" wrote:
> I am looking to get a second opinion about disabling hinted-handoffs. I
> have an application that can tolerate a fair amount of inconsistency
> (advertising domain), and so I'm weighting the pros and cons of hinted
> handoffs. I'm
intsColumnFamily
consists of
>> one row (Rowkey 00, nothing more). The "problem" started after
upgrading
>> from 1.1.4 to 1.1.6. Every ten minutes HintedHandoffManager starts
and
>> finishes after sending "0 rows".
>>
>> .vegard,
>>
>&
more). The "problem" started after upgrading
>> from 1.1.4 to 1.1.6. Every ten minutes HintedHandoffManager starts and
>> finishes after sending "0 rows".
>>
>> .vegard,
>>
>>
>>
>> - Original Message -
>> From:
>&
;.
>
> .vegard,
>
>
>
> - Original Message -
> From:
> user@cassandra.apache.org
>
> To:
>
> Cc:
>
> Sent:
> Mon, 29 Oct 2012 23:45:30 +0100
>
> Subject:
> Re: Hinted Handoff runs every ten minutes
>
>
> Dne 29.10.2012 23:24, Steph
ncated the column family:
[default@system] truncate HintsColumnFamily;
HintsColumnFamily truncated.
-Original Message-
From: Radim Kolar [mailto:h...@filez.com]
Sent: Monday, October 29, 2012 3:46 PM
To: user@cassandra.apache.org
Subject: Re: Hinted Handoff runs every ten minutes
Dne 29.
Hi,
I have the exact same problem with 1.1.6. HintsColumnFamily consists
of one row (Rowkey 00, nothing more). The "problem" started after
upgrading from 1.1.4 to 1.1.6. Every ten minutes
HintedHandoffManager starts and finishes after sending "0 rows".
.vegard,
- Original Message -
Hi,
I have the exact same problem with 1.1.6. HintsColumnFamily consists
of one row (Rowkey 00, nothing more). The "problem" started after
upgrading from 1.1.4 to 1.1.6. Every ten minutes
HintedHandoffManager starts and finishes after sending "0 rows".
.vegard,
- Original Message -
Dne 29.10.2012 23:24, Stephen Pierce napsal(a):
I'm running 1.1.5; the bug says it's fixed in 1.0.9/1.1.0.
How can I check to see why it keeps running HintedHandoff?
you have tombstone is system.HintsColumnFamily use list command in
cassandra-cli to check
I'm running 1.1.5; the bug says it's fixed in 1.0.9/1.1.0.
How can I check to see why it keeps running HintedHandoff?
Steve
-Original Message-
From: Brandon Williams [mailto:dri...@gmail.com]
Sent: Wednesday, October 24, 2012 4:56 AM
To: user@cassandra.apache.org
Subject:
> With both data centers functional, the test takes just a few minutes to run,
> with one data center down, 15x the amount of time.
Could you provide the numbers, it's easier to get a feel for how the throughput
is dropping. Does latency reported by nodetool cf stats change ?
I'm also interested
On Oct 24, 2012, at 6:05 PM, aaron morton wrote:
> Hints store the columns, row key, KS name and CF id(s) for each mutation to
> each node. Where an executed mutation will store the most recent columns
> collated with others under the same row key. So depending on the type of
> mutation hints
Hints store the columns, row key, KS name and CF id(s) for each mutation to
each node. Where an executed mutation will store the most recent columns
collated with others under the same row key. So depending on the type of
mutation hints will take up more space.
The worse case would be lots of
Thanks.
I thought it had been addressed so before but couldn't find the ticket.
Cheers
-
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com
On 25/10/2012, at 12:56 AM, Brandon Williams wrote:
> On Sun, Oct 21, 2012 at 6:44 PM, aaron morton wrote:
>> I
Is there a walk around other than upgrade?
Thanks,
*Tamar Fraenkel *
Senior Software Engineer, TOK Media
[image: Inline image 1]
ta...@tok-media.com
Tel: +972 2 6409736
Mob: +972 54 8356490
Fax: +972 2 5612956
On Wed, Oct 24, 2012 at 1:56 PM, Brandon Williams wrote:
> On Sun, Oct 21,
On Sun, Oct 21, 2012 at 6:44 PM, aaron morton wrote:
> I *think* this may be ghost rows which have not being compacted.
You would be correct in the case of 1.0.8:
https://issues.apache.org/jira/browse/CASSANDRA-3955
-Brandon
Hi!
I am having the same issue on 1.0.8.
Checked number of SSTables, on two nodes I have 1 (on each) and on 1 node I
have none.
Thanks,
*Tamar Fraenkel *
Senior Software Engineer, TOK Media
[image: Inline image 1]
ta...@tok-media.com
Tel: +972 2 6409736
Mob: +972 54 8356490
Fax: +972 2 5612
I *think* this may be ghost rows which have not being compacted.
How many SSTables are on disk for the HintedHandoff CF ?
Cheers
-
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com
On 19/10/2012, at 7:16 AM, David Daeschler wrote:
> Hi Steve,
>
> Also
Hi Steve,
Also confirming this. After having a node go down on Cassandra 1.0.8
there seems to be hinted handoff between two of our 4 nodes every 10
minutes. Our setup also shows 0 rows. It does not appear to have any
effect on the operation of the ring, just fills up the log files.
- David
On
Right.
See also https://issues.apache.org/jira/browse/CASSANDRA-3958.
On Thu, Feb 23, 2012 at 5:43 PM, Todd Burruss wrote:
> if I remember correctly, cassandra has a random delay in it so hint
> deliver is staggered and does not overwhelm the just restarted node.
>
> On 2/23/12 1:46 PM, "Hontvár
I've verified it in the source: deliverHintsToEndpointInternal in
HintedHandOffManager.java
Yes it add random delay before HH delivery.
2012/2/24 Todd Burruss :
> if I remember correctly, cassandra has a random delay in it so hint
> deliver is staggered and does not overwhelm the just restarted no
if I remember correctly, cassandra has a random delay in it so hint
deliver is staggered and does not overwhelm the just restarted node.
On 2/23/12 1:46 PM, "Hontvári József Levente"
wrote:
>I have played with a test cluster, stopping cassandra on one node and
>updating a row on another. I notic
Nope, that's a separate issue.
https://issues.apache.org/jira/browse/CASSANDRA-3554
On Thu, Dec 1, 2011 at 5:59 PM, Terje Marthinussen
wrote:
> Sorry for not checking source to see if things have changed but i just
> remembered an issue I have forgotten to make jira for.
>
> In old days, nodes w
Sorry for not checking source to see if things have changed but i just
remembered an issue I have forgotten to make jira for.
In old days, nodes would periodically try to deliver queues.
However, this was at some stage changed so it only deliver if a node is being
marked up.
However, you can d
Yes, I'll do that.
/Fredrik
Sylvain Lebresne skrev 2011-12-01 11:10:
You're right, good catch.
Do you mind opening a ticket on jira
(https://issues.apache.org/jira/browse/CASSANDRA)?
--
Sylvain
On Thu, Dec 1, 2011 at 10:03 AM, Fredrik L Stigbäck
wrote:
Hi,
We,re running cassandra 1.0.3.
I'v
You're right, good catch.
Do you mind opening a ticket on jira
(https://issues.apache.org/jira/browse/CASSANDRA)?
--
Sylvain
On Thu, Dec 1, 2011 at 10:03 AM, Fredrik L Stigbäck
wrote:
> Hi,
> We,re running cassandra 1.0.3.
> I've done some testing with 2 nodes (node A, node B), replication facto
I missed that one in the release notes. Thank you!
-Original Message-
From: Jonathan Ellis [mailto:jbel...@gmail.com]
Sent: vrijdag 7 oktober 2011 16:39
To: user@cassandra.apache.org
Subject: Re: Hinted handoff question
This was fixed in 0.7.8 by CASSANDRA-2928
In general it's a
we have 0.7.7. in production.
>
>
>
> Thanks!
>
>
>
> Rene
>
>
>
> From: aaron morton [mailto:aa...@thelastpickle.com]
> Sent: donderdag 6 oktober 2011 22:51
>
> To: user@cassandra.apache.org
> Subject: Re: Hinted handoff question
>
>
>
> At debug
. Unfortunately,
we have 0.7.7. in production.
Thanks!
Rene
From: aaron morton [mailto:aa...@thelastpickle.com]
Sent: donderdag 6 oktober 2011 22:51
To: user@cassandra.apache.org
Subject: Re: Hinted handoff question
At debug level you should see some log messages such as...
"Checking r
> From: aaron morton [mailto:aa...@thelastpickle.com]
> Sent: donderdag 6 oktober 2011 12:13
> To: user@cassandra.apache.org
> Subject: Re: Hinted handoff question
>
> Check that node 1 sees node 3 as UP (via ring).
>
> Check the tpstats on node 1, is there an active HH
org
Subject: Re: Hinted handoff question
Check that node 1 sees node 3 as UP (via ring).
Check the tpstats on node 1, is there an active HH task ?
Take another crawl through the logs.
Cheers
-
Aaron Morton
Freelance Cassandra Developer
@aaronmorton
http://www.thelastpickle.com
On 6
Check that node 1 sees node 3 as UP (via ring).
Check the tpstats on node 1, is there an active HH task ?
Take another crawl through the logs.
Cheers
-
Aaron Morton
Freelance Cassandra Developer
@aaronmorton
http://www.thelastpickle.com
On 6/10/2011, at 10:35 PM, Rene Kochen
(1) you should upgrade to a stable release instead of a frankenbuild
(2) hh_enabled just controls whether it creates new hints. it will
still attempt to deliver existing ones. this is essentially free if
no such hints exist.
tldr: it's harmless.
On Thu, Jun 23, 2011 at 4:55 PM, Jeffrey Wang w
bject: Re: hinted handoff sleeping
On Thu, Jun 23, 2011 at 2:55 PM, Jeffrey Wang wrote:
> Hey all,
>
>
>
> We’re running a slightly patched version of 0.7.3 on a cluster of 5 nodes.
> I’ve been noticing a number of messages in our logs which look like this
> (after a node goe
On Thu, Jun 23, 2011 at 2:55 PM, Jeffrey Wang wrote:
> Hey all,
>
>
>
> We’re running a slightly patched version of 0.7.3 on a cluster of 5 nodes.
> I’ve been noticing a number of messages in our logs which look like this
> (after a node goes “down” and comes back up, usually just due to a GC):
>
HH will be stored into one of live replica node. It is just a hint,
rather than data to be replicated.
maki
2011/5/12 Anurag Gujral :
> Hi All,
> I have two questions:
> a) Is there a way to turn on and off hinted handoff per keyspace rather
> than for multiple keyspaces.
> b)It looks
I'm not sure about your first question.
I believe the internal system keyspace holds the hinted handoff information.
In 0.6 and earlier, HintedHandoffManager.sendMessage used to read the entire
row into memory and then send the row back to the client in a single
message. As of 0.7, Cassandra page
60 matches
Mail list logo