Re: Is my range read query behaving strange ?

2019-06-13 Thread Laxmikant Upadhyay
Raised a ticket  https://issues.apache.org/jira/browse/CASSANDRA-15159 for
the same.

On Thu, Jun 13, 2019 at 3:55 PM Laxmikant Upadhyay 
wrote:

> This issue is reproducible on *3.11.4 and 2.1.21* as well.  (not yet
> checked on  3.0)
>
> Range query could be : select * from test.table1;  (In this case, Read
> repair actually sending old mutation to the node which has tombstone  )
> I also ran normal read and it also returns a row in this case instead of
> empty response.
>
> cqlsh> select * from test.table1;
>
>  key  | col | val
> --+-+-
>  key2 | abc | xyz
>
> (1 rows)
> cqlsh> select * from test.table1 where key = 'key2';
>
>  key  | col | val
> --+-+-
>  key2 | abc | xyz
>
> Why is this behaviour ?  Why read is ignoring purge-able tombstone ?
>
>
>
> On Thu, Jun 13, 2019 at 12:30 PM Laxmikant Upadhyay <
> laxmikant@gmail.com> wrote:
>
>> HI Michael,
>>
>> Thanks for your reply.
>> I don't think this issue is related to CASSANDRA-12765
>>  as in my case
>> the sstable which has tombstone does not have maxLocalDeletionTime ==
>> Integer.MAX_VALUE .  I am able to reproduce this issue on 2.1.17 as well.
>>
>> I am attaching the steps to reproduce on 2.1.17 (with minor change from
>> previous steps to make sure one request must go to the node which has old
>> mutation). I have also attached the trace of range read query.
>>
>> Should I raise a jira for the same ?
>>
>> On Wed, Jun 12, 2019 at 9:00 AM Michael Shuler 
>> wrote:
>>
>>> (dropped dev@ x-post; user@ was correct)
>>>
>>> Possibly #12765, fixed in 2.1.17. Wouldn't hurt to update to latest
>>> 2.1.21.
>>>
>>> https://issues.apache.org/jira/browse/CASSANDRA-12765
>>> https://github.com/apache/cassandra/blob/cassandra-2.1/CHANGES.txt#L1-L36
>>>
>>> Michael
>>>
>>> On 6/11/19 9:58 PM, Laxmikant Upadhyay wrote:
>>> > Does range query ignore purgable tombstone (which crossed grace
>>> period)
>>> > in some cases?
>>> >
>>> > On Tue, Jun 11, 2019, 2:56 PM Laxmikant Upadhyay
>>> > mailto:laxmikant@gmail.com>> wrote:
>>> >
>>> > In a 3 node cassandra 2.1.16 cluster where, one node has old
>>> > mutation and two nodes have evict-able (crossed gc grace period)
>>> > tombstone produced by TTL. A read range  query with local quorum
>>> > return the old mutation as result. However expected result should
>>> be
>>> > empty. Next time running the same query results no data as
>>> expected.
>>> > Why this strange behaviour?
>>> >
>>> >
>>> > *Steps to Reproduce :*
>>> > Create a cassandra-2.1.16  3 node cluster. Disable hinted handoff
>>> > for each node.
>>> >
>>> > #ccm node1 nodetool ring
>>> > Datacenter: datacenter1
>>> > ==
>>> > AddressRackStatus State   LoadOwns
>>>
>>> >   Token
>>> >
>>>
>>> >3074457345618258602
>>> > 127.0.0.1  rack1   Up Normal  175.12 KB   100.00%
>>> >-9223372036854775808
>>> > 127.0.0.2  rack1   Up   Normal  177.87 KB   100.00%
>>> >  -3074457345618258603
>>> > 127.0.0.3  rack1   Up   Normal  175.13 KB   100.00%
>>> >  3074457345618258602
>>> >
>>> > #Connect to cqlsh and set CONISISTENCY LOCAL_QUORUM;
>>> >
>>> > cqlsh> CREATE KEYSPACE IF NOT EXISTS test WITH REPLICATION = {
>>> > 'class' : 'NetworkTopologyStrategy', 'datacenter1' : 3 };
>>> > cqlsh> CREATE TABLE test.table1 (key text, col text, val
>>> > text,PRIMARY KEY ((key), col));
>>> > cqlsh> ALTER TABLE test.table1  with GC_GRACE_SECONDS = 120;
>>> >
>>> > cqlsh> INSERT INTO test.table1  (key, col, val) VALUES ('key2',
>>> > 'abc','xyz');
>>> >
>>> > #ccm flush
>>> >
>>> > #ccm node3 stop
>>> >
>>> > cqlsh> INSERT INTO test.table1  (key, col, val) VALUES ('key2',
>>> > 'abc','xyz') USING TTL 60;
>>> >
>>> > #ccm flush
>>> >
>>> > #wait for 3 min so that the tombstone crosses its gc grace period.
>>> >
>>> > #ccm node3 start
>>> >
>>> > cqlsh> select * from test.table1 where token (key) >
>>> > 3074457345618258602 and token (key) < -9223372036854775808 ;
>>> >
>>> >   key  | col | val
>>> > --+-+-
>>> >   key2 | abc | xyz
>>> >
>>> > (1 rows)
>>> >
>>> > #ccm flush
>>> > -> Here read repair triggers and the old mutation moves to the one
>>> > of the node where tombstone is present (not both the node)
>>> >
>>> >
>>> > cqlsh> select * from test.vouchers where token (key) >
>>> > 3074457345618258602 and token (key) < -9223372036854775808 ;
>>> >
>>> >   key | col | val
>>> > -+-+-
>>> >
>>> > (0 rows)
>>> >
>>> >
>>> > --
>>> >
>>> > regards,
>>> > Laxmikant Upadhyay
>>> >
>>>
>>>
>>> -
>>> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
>>> For additional commands, e-mail: 

Re: Recover lost node from backup or evict/re-add?

2019-06-13 Thread Jeff Jirsa


> On Jun 13, 2019, at 6:29 AM, Oleksandr Shulgin  
> wrote:
> 
>> On Thu, Jun 13, 2019 at 3:16 PM Jeff Jirsa  wrote:
> 
>> On Jun 13, 2019, at 2:52 AM, Oleksandr Shulgin 
>>  wrote:
>> On Wed, Jun 12, 2019 at 4:02 PM Jeff Jirsa  wrote:
 To avoid violating consistency guarantees, you have to repair the replicas 
 while the lost node is down
>>> 
>>> How do you suggest to trigger it?  Potentially replicas of the primary 
>>> range for the down node are all over the local DC, so I would go with 
>>> triggering a full cluster repair with Cassandra Reaper.  But isn't it going 
>>> to fail because of the down node?  
>> Im not sure there’s an easy and obvious path here - this is something TLP 
>> may want to enhance reaper to help with. 
>> 
>> You have to specify the ranges with -st/-et, and you have to tell it to 
>> ignore the down host with -hosts. With vnodes you’re right that this may be 
>> lots and lots of ranges all over the ring.
>> 
>> There’s a patch proposed (maybe committed in 4.0) that makes this a nonissue 
>> by allowing bootstrap to stream one repaired set and all of the unrepaired 
>> replica data (which is probably very small if you’re running IR regularly), 
>> which accomplished the same thing.
> 
> Ouch, it really hurts to learn this. :(
>>> It is also documented (I believe) that one should repair the node after it 
>>> finishes the "replace address" procedure.  So should one repair before and 
>>> after?
>> You do not need to repair after the bootstrap if you repair before. If the 
>> docs say that, they’re wrong. The joining host gets writes during bootstrap 
>> and consistency levels are altered during bootstrap to account for the 
>> joining host.
> 
> This is what I had in mind (what makes replacement different from actual 
> bootstrap of a new node):

Bootstrapping a new node does not require repairs at all.

Replacing a node only requires repairs to guarantee consistency to avoid 
violating quorum because streaming for bootstrap only streams from one replica

Think this way:

Host 1, 2, 3 in a replica set
You write value A to some key
It lands on hosts 1 and 3. Host 2 was being restarted or something
Host 2 comes back up
Host 3 fails

If you replace 3 with 3’ - 
3’ May stream from host 1 and now you’ve got a quorum if replicas with A
3’ may stream fr host 2, and now you’ve got a quorum if replicas without A. 
This is illegal.

This is just a statistics game - do you have hosts missing writes? If so, are 
hints delivering them when those hosts come back? What’s the cost of violating 
consistency in that second scenario to you? 

If you’re running something where correctness really really really matters, you 
must repair first. If you’re actually running a truly eventual consistency use 
case and reading stale writes is fine, you probably won’t ever notice.  

In any case these docs are weird and wrong - joining nodes get writes in all 
versions of Cassandra for the past few years (at least 2.0+), so the docs 
really need to be fixed.

> http://cassandra.apache.org/doc/latest/operating/topo_changes.html?highlight=replace%20address#replacing-a-dead-node
>  
> Note
> If any of the following cases apply, you MUST run repair to make the replaced 
> node consistent again, since it missed ongoing writes during/prior to 
> bootstrapping. The replacement timeframe refers to the period from when the 
> node initially dies to when a new node completes the replacement process.
> 
> The node is down for longer than max_hint_window_in_ms before being replaced.
> You are replacing using the same IP address as the dead node and replacement 
> takes longer than max_hint_window_in_ms.
> 
> I would imagine that any production size instance would take way longer to 
> replace than the default max hint window (which is 3 hours, AFAIK).  Didn't 
> remember the same IP restriction, but at least this I would also expect to be 
> the most common setup.
> 
> --
> Alex
> 


Re: Recover lost node from backup or evict/re-add?

2019-06-13 Thread Oleksandr Shulgin
On Thu, Jun 13, 2019 at 3:16 PM Jeff Jirsa  wrote:

> On Jun 13, 2019, at 2:52 AM, Oleksandr Shulgin <
> oleksandr.shul...@zalando.de> wrote:
> On Wed, Jun 12, 2019 at 4:02 PM Jeff Jirsa  wrote:
>
> To avoid violating consistency guarantees, you have to repair the replicas
>> while the lost node is down
>>
>
> How do you suggest to trigger it?  Potentially replicas of the primary
> range for the down node are all over the local DC, so I would go with
> triggering a full cluster repair with Cassandra Reaper.  But isn't it going
> to fail because of the down node?
>
> Im not sure there’s an easy and obvious path here - this is something TLP
> may want to enhance reaper to help with.
>
> You have to specify the ranges with -st/-et, and you have to tell it to
> ignore the down host with -hosts. With vnodes you’re right that this may be
> lots and lots of ranges all over the ring.
>
> There’s a patch proposed (maybe committed in 4.0) that makes this a
> nonissue by allowing bootstrap to stream one repaired set and all of the
> unrepaired replica data (which is probably very small if you’re running IR
> regularly), which accomplished the same thing.
>

Ouch, it really hurts to learn this. :(

> It is also documented (I believe) that one should repair the node after it
> finishes the "replace address" procedure.  So should one repair before and
> after?
>
> You do not need to repair after the bootstrap if you repair before. If the
> docs say that, they’re wrong. The joining host gets writes during bootstrap
> and consistency levels are altered during bootstrap to account for the
> joining host.
>

This is what I had in mind (what makes replacement different from actual
bootstrap of a new node):
http://cassandra.apache.org/doc/latest/operating/topo_changes.html?highlight=replace%20address#replacing-a-dead-node


Note

If any of the following cases apply, you MUST run repair to make the replaced
node consistent again, since it missed ongoing writes during/prior to
bootstrapping. The *replacement* timeframe refers to the period from when
the node initially dies to when a new node completes the replacement
process.


   1. The node is down for longer than max_hint_window_in_ms before being
  replaced.
  2. You are replacing using the same IP address as the dead node and
  replacement takes longer than max_hint_window_in_ms.


I would imagine that any production size instance would take way longer to
replace than the default max hint window (which is 3 hours, AFAIK).  Didn't
remember the same IP restriction, but at least this I would also expect to
be the most common setup.

--
Alex


Re: Recover lost node from backup or evict/re-add?

2019-06-13 Thread Jeff Jirsa


> On Jun 13, 2019, at 2:52 AM, Oleksandr Shulgin  
> wrote:
> 
>> On Wed, Jun 12, 2019 at 4:02 PM Jeff Jirsa  wrote:
> 
>> To avoid violating consistency guarantees, you have to repair the replicas 
>> while the lost node is down
> 
> How do you suggest to trigger it?  Potentially replicas of the primary range 
> for the down node are all over the local DC, so I would go with triggering a 
> full cluster repair with Cassandra Reaper.  But isn't it going to fail 
> because of the down node?  

Im not sure there’s an easy and obvious path here - this is something TLP may 
want to enhance reaper to help with. 

You have to specify the ranges with -st/-et, and you have to tell it to ignore 
the down host with -hosts. With vnodes you’re right that this may be lots and 
lots of ranges all over the ring.

There’s a patch proposed (maybe committed in 4.0) that makes this a nonissue by 
allowing bootstrap to stream one repaired set and all of the unrepaired replica 
data (which is probably very small if you’re running IR regularly), which 
accomplished the same thing.

> 
> It is also documented (I believe) that one should repair the node after it 
> finishes the "replace address" procedure.  So should one repair before and 
> after?

You do not need to repair after the bootstrap if you repair before. If the docs 
say that, they’re wrong. The joining host gets writes during bootstrap and 
consistency levels are altered during bootstrap to account for the joining host.

Re: very slow repair

2019-06-13 Thread Oleksandr Shulgin
On Thu, Jun 13, 2019 at 2:09 PM Léo FERLIN SUTTON
 wrote:

> Last, but not least: are you using the default number of vnodes, 256?  The
>> overhead of large number of vnodes (times the number of nodes), can be
>> quite significant.  We've seen major improvements in repair runtime after
>> switching from 256 to 16 vnodes on Cassandra version 3.0.
>
>
> Is there a recommended procedure to switch the amount of vnodes ?
>

Yes.  One should deploy a new virtual DC with desired configuration and
rebuild from the original one, then decommission the old virtual DC.

With the smaller number of vnodes you should use
allocate_tokens_for_keyspace configuration parameter to ensure uniform load
distribution.  The caveat is that the nodes allocate tokens before they
bootstrap, so the very first nodes will not have keyspace information
available.  This can be worked around, though it is not trivial.  See this
thread for our past experience:
https://lists.apache.org/thread.html/396f2d20397c36b9cff88a0c2c5523154d420ece24a4dafc9fde3d1f@%3Cuser.cassandra.apache.org%3E

--
Alex


Re: Speed up compaction

2019-06-13 Thread Alexander Dejanovski
Hi Léo,

Major compactions in LCS (and minor as well) are very slow indeed and I'm
afraid there's not much you can do to speed things up. There are lots of
synchronized sections in the LCS code and it has to do a lot of comparisons
between sstables to make sure a partition won't end up in two sstables of
the same level.
A major compaction will be single threaded for obvious reasons, and while
this is happening you might have all the newly flushed SSTables that will
pile up in S0 since I don't see how Cassandra could achieve the "one
sstable per partition per level except L0" guarantee otherwise.

At this point, your best chance might be to switch the table to STCS, run a
major compaction using the "-s" flag (split output, which will create one
SSTable per size tier instead of a big fat one) and then back to LCS,
before or after your migration (whatever works best for you). If you go
down that path, I'd also recommend to try it up on one node using JMX to
alter the compaction strategy, run the major compaction with nodetool and
see if it's indeed faster than the LCS major compaction. Then, proceed node
by node using JMX (wait for the major compaction to go through between
nodes) and alter the schema only after the last node has been switched to
STCS.
You can use more "aggressive" compaction settings to limit read
fragmentation reducing max_threshold to 3 instead of 4 (the default).

Note that doing all this will impact your cluster performance in ways I
cannot predict, and should be attempted only if you really need to perform
this major compaction and cannot wait for it to go through at the current
pace.

Cheers,

-
Alexander Dejanovski
France
@alexanderdeja

Consultant
Apache Cassandra Consulting
http://www.thelastpickle.com


On Thu, Jun 13, 2019 at 2:07 PM Léo FERLIN SUTTON
 wrote:

> On Thu, Jun 13, 2019 at 12:09 PM Oleksandr Shulgin <
> oleksandr.shul...@zalando.de> wrote:
>
>> On Thu, Jun 13, 2019 at 11:28 AM Léo FERLIN SUTTON
>>  wrote:
>>
>>>
>>> ## Cassandra configuration :
>>> 4 concurrent_compactors
>>> Current compaction throughput: 150 MB/s
>>> Concurrent reads/write are both set to 128.
>>>
>>> I have also temporarily stopped every repair operations.
>>>
>>> Any ideas about how I can speed this up ?
>>>
>>
>> Hi,
>>
>> What is the compaction strategy used by this column family?
>>
>> Do you observe this behavior on one of the nodes only?  Have you tried to
>> cancel this compaction and see if a new one is started and makes better
>> progress?  Can you try to restart the affected node?
>>
>> Regards,
>> --
>> Alex
>>
>> I can't believe I forgot that information.
>
>  Overall we are talking about a 1.08TB table, using LCS.
>
> SSTable count: 1047
>> SSTables in each level: [15/4, 10, 103/100, 918, 0, 0, 0, 0, 0]
>
> SSTable Compression Ratio: 0.5192269874287099
>
> Number of partitions (estimate): 7282253587
>
>
> We have recently (about a month ago) deleted about 25% of the data in that
> table.
>
> Letting Cassandra reclaim the disk space on it's own (via regular
> compactions) was too slow for us, so we wanted to force a compaction on the
> table to reclaim the disk space faster.
>
> The speed of the compaction doesn't seem out of the ordinary for the
> cluster, only before we haven't had such a big compaction and the speed
> alarmed us.
> We never have a big compaction backlog, most of the time less than 5
> pending tasks (per node)
>
> Finally but we are running Cassandra 3.0.18 and plan to upgrade to 3.11 as
> soon as our compactions are over.
>
> Regards,
>
> Leo
>


Re: Speed up compaction

2019-06-13 Thread Léo FERLIN SUTTON
On Thu, Jun 13, 2019 at 2:44 PM Oleksandr Shulgin <
oleksandr.shul...@zalando.de> wrote:

> On Thu, Jun 13, 2019 at 2:07 PM Léo FERLIN SUTTON
>  wrote:
>
>>
>>  Overall we are talking about a 1.08TB table, using LCS.
>>
>> SSTable count: 1047
>>> SSTables in each level: [15/4, 10, 103/100, 918, 0, 0, 0, 0, 0]
>>
>> SSTable Compression Ratio: 0.5192269874287099
>>
>> Number of partitions (estimate): 7282253587
>>
>>
>> We have recently (about a month ago) deleted about 25% of the data in
>> that table.
>>
>> Letting Cassandra reclaim the disk space on it's own (via regular
>> compactions) was too slow for us, so we wanted to force a compaction on the
>> table to reclaim the disk space faster.
>>
>
> To be clear, that compaction task is running the major compaction for this
> column family?  I have no experience with Leveled compaction strategy, so
> not really sure what behavior to expect from it.  I can imagine that with
> that many SSTables and a major compaction, there might be quite some
> overhead as it does more than an ordinary merge-sort as I would expect from
> Size-tiered.
>

Yes It is running a major compaction.

Even with a lot of overhead, we expect it to take more than two weeks :( We
are hoping to find some way to speed up things a bit.


Re: Speed up compaction

2019-06-13 Thread Oleksandr Shulgin
On Thu, Jun 13, 2019 at 2:07 PM Léo FERLIN SUTTON
 wrote:

>
>  Overall we are talking about a 1.08TB table, using LCS.
>
> SSTable count: 1047
>> SSTables in each level: [15/4, 10, 103/100, 918, 0, 0, 0, 0, 0]
>
> SSTable Compression Ratio: 0.5192269874287099
>
> Number of partitions (estimate): 7282253587
>
>
> We have recently (about a month ago) deleted about 25% of the data in that
> table.
>
> Letting Cassandra reclaim the disk space on it's own (via regular
> compactions) was too slow for us, so we wanted to force a compaction on the
> table to reclaim the disk space faster.
>

To be clear, that compaction task is running the major compaction for this
column family?  I have no experience with Leveled compaction strategy, so
not really sure what behavior to expect from it.  I can imagine that with
that many SSTables and a major compaction, there might be quite some
overhead as it does more than an ordinary merge-sort as I would expect from
Size-tiered.

--
Alex


Re: very slow repair

2019-06-13 Thread Léo FERLIN SUTTON
>
> Last, but not least: are you using the default number of vnodes, 256?  The
> overhead of large number of vnodes (times the number of nodes), can be
> quite significant.  We've seen major improvements in repair runtime after
> switching from 256 to 16 vnodes on Cassandra version 3.0.


Is there a recommended procedure to switch the amount of vnodes ?

Regards,

Leo

On Thu, Jun 13, 2019 at 12:06 PM Oleksandr Shulgin <
oleksandr.shul...@zalando.de> wrote:

> On Thu, Jun 13, 2019 at 10:36 AM R. T. 
> wrote:
>
>>
>> Well, actually by running cfstats I can see that the totaldiskspaceused
>> is about ~ 1.2 TB per node in the DC1 and ~ 1 TB per node in DC2. DC2 was
>> off for a while thats why there is a difference in space.
>>
>> I am using Cassandra 3.0.6 and
>> my stream_throughput_outbound_megabits_per_sec is th4e default setting so
>> according to my version is (200 Mbps or 25 MB/s)
>>
>
> And the other setting: compaction_throughput_mb_per_sec?  It is also
> highly relevant for repair performance, as streamed in files need to be
> compacted with the existing files on the nodes.  In our experience change
> in compaction throughput limit is almost linearly reflected by the repair
> run time.
>
> The default 16 MB/s is too limiting for any production grade setup, I
> believe.  We go as high as 90 MB/s on AWS EBS gp2 data volumes.  But don't
> take it as a gospel, I'd suggest you start increasing the setting (e.g. by
> doubling it) and observe how it affects repair performance (and client
> latencies).
>
> Have you tried with "parallel" instead of "DC parallel" mode?  The latter
> one is really poorly named and it actually means something else, as neatly
> highlighted in this SO answer: https://dba.stackexchange.com/a/175028
>
> Last, but not least: are you using the default number of vnodes, 256?  The
> overhead of large number of vnodes (times the number of nodes), can be
> quite significant.  We've seen major improvements in repair runtime after
> switching from 256 to 16 vnodes on Cassandra version 3.0.
>
> Cheers,
> --
> Alex
>
>


Re: Speed up compaction

2019-06-13 Thread Léo FERLIN SUTTON
On Thu, Jun 13, 2019 at 12:09 PM Oleksandr Shulgin <
oleksandr.shul...@zalando.de> wrote:

> On Thu, Jun 13, 2019 at 11:28 AM Léo FERLIN SUTTON
>  wrote:
>
>>
>> ## Cassandra configuration :
>> 4 concurrent_compactors
>> Current compaction throughput: 150 MB/s
>> Concurrent reads/write are both set to 128.
>>
>> I have also temporarily stopped every repair operations.
>>
>> Any ideas about how I can speed this up ?
>>
>
> Hi,
>
> What is the compaction strategy used by this column family?
>
> Do you observe this behavior on one of the nodes only?  Have you tried to
> cancel this compaction and see if a new one is started and makes better
> progress?  Can you try to restart the affected node?
>
> Regards,
> --
> Alex
>
> I can't believe I forgot that information.

 Overall we are talking about a 1.08TB table, using LCS.

SSTable count: 1047
> SSTables in each level: [15/4, 10, 103/100, 918, 0, 0, 0, 0, 0]

SSTable Compression Ratio: 0.5192269874287099

Number of partitions (estimate): 7282253587


We have recently (about a month ago) deleted about 25% of the data in that
table.

Letting Cassandra reclaim the disk space on it's own (via regular
compactions) was too slow for us, so we wanted to force a compaction on the
table to reclaim the disk space faster.

The speed of the compaction doesn't seem out of the ordinary for the
cluster, only before we haven't had such a big compaction and the speed
alarmed us.
We never have a big compaction backlog, most of the time less than 5
pending tasks (per node)

Finally but we are running Cassandra 3.0.18 and plan to upgrade to 3.11 as
soon as our compactions are over.

Regards,

Leo


Re: Is my range read query behaving strange ?

2019-06-13 Thread Laxmikant Upadhyay
This issue is reproducible on *3.11.4 and 2.1.21* as well.  (not yet
checked on  3.0)

Range query could be : select * from test.table1;  (In this case, Read
repair actually sending old mutation to the node which has tombstone  )
I also ran normal read and it also returns a row in this case instead of
empty response.

cqlsh> select * from test.table1;

 key  | col | val
--+-+-
 key2 | abc | xyz

(1 rows)
cqlsh> select * from test.table1 where key = 'key2';

 key  | col | val
--+-+-
 key2 | abc | xyz

Why is this behaviour ?  Why read is ignoring purge-able tombstone ?



On Thu, Jun 13, 2019 at 12:30 PM Laxmikant Upadhyay 
wrote:

> HI Michael,
>
> Thanks for your reply.
> I don't think this issue is related to CASSANDRA-12765
>  as in my case the
> sstable which has tombstone does not have maxLocalDeletionTime ==
> Integer.MAX_VALUE .  I am able to reproduce this issue on 2.1.17 as well.
>
> I am attaching the steps to reproduce on 2.1.17 (with minor change from
> previous steps to make sure one request must go to the node which has old
> mutation). I have also attached the trace of range read query.
>
> Should I raise a jira for the same ?
>
> On Wed, Jun 12, 2019 at 9:00 AM Michael Shuler 
> wrote:
>
>> (dropped dev@ x-post; user@ was correct)
>>
>> Possibly #12765, fixed in 2.1.17. Wouldn't hurt to update to latest
>> 2.1.21.
>>
>> https://issues.apache.org/jira/browse/CASSANDRA-12765
>> https://github.com/apache/cassandra/blob/cassandra-2.1/CHANGES.txt#L1-L36
>>
>> Michael
>>
>> On 6/11/19 9:58 PM, Laxmikant Upadhyay wrote:
>> > Does range query ignore purgable tombstone (which crossed grace period)
>> > in some cases?
>> >
>> > On Tue, Jun 11, 2019, 2:56 PM Laxmikant Upadhyay
>> > mailto:laxmikant@gmail.com>> wrote:
>> >
>> > In a 3 node cassandra 2.1.16 cluster where, one node has old
>> > mutation and two nodes have evict-able (crossed gc grace period)
>> > tombstone produced by TTL. A read range  query with local quorum
>> > return the old mutation as result. However expected result should be
>> > empty. Next time running the same query results no data as expected.
>> > Why this strange behaviour?
>> >
>> >
>> > *Steps to Reproduce :*
>> > Create a cassandra-2.1.16  3 node cluster. Disable hinted handoff
>> > for each node.
>> >
>> > #ccm node1 nodetool ring
>> > Datacenter: datacenter1
>> > ==
>> > AddressRackStatus State   LoadOwns
>> >   Token
>> >
>> >3074457345618258602
>> > 127.0.0.1  rack1   Up Normal  175.12 KB   100.00%
>> >-9223372036854775808
>> > 127.0.0.2  rack1   Up   Normal  177.87 KB   100.00%
>> >  -3074457345618258603
>> > 127.0.0.3  rack1   Up   Normal  175.13 KB   100.00%
>> >  3074457345618258602
>> >
>> > #Connect to cqlsh and set CONISISTENCY LOCAL_QUORUM;
>> >
>> > cqlsh> CREATE KEYSPACE IF NOT EXISTS test WITH REPLICATION = {
>> > 'class' : 'NetworkTopologyStrategy', 'datacenter1' : 3 };
>> > cqlsh> CREATE TABLE test.table1 (key text, col text, val
>> > text,PRIMARY KEY ((key), col));
>> > cqlsh> ALTER TABLE test.table1  with GC_GRACE_SECONDS = 120;
>> >
>> > cqlsh> INSERT INTO test.table1  (key, col, val) VALUES ('key2',
>> > 'abc','xyz');
>> >
>> > #ccm flush
>> >
>> > #ccm node3 stop
>> >
>> > cqlsh> INSERT INTO test.table1  (key, col, val) VALUES ('key2',
>> > 'abc','xyz') USING TTL 60;
>> >
>> > #ccm flush
>> >
>> > #wait for 3 min so that the tombstone crosses its gc grace period.
>> >
>> > #ccm node3 start
>> >
>> > cqlsh> select * from test.table1 where token (key) >
>> > 3074457345618258602 and token (key) < -9223372036854775808 ;
>> >
>> >   key  | col | val
>> > --+-+-
>> >   key2 | abc | xyz
>> >
>> > (1 rows)
>> >
>> > #ccm flush
>> > -> Here read repair triggers and the old mutation moves to the one
>> > of the node where tombstone is present (not both the node)
>> >
>> >
>> > cqlsh> select * from test.vouchers where token (key) >
>> > 3074457345618258602 and token (key) < -9223372036854775808 ;
>> >
>> >   key | col | val
>> > -+-+-
>> >
>> > (0 rows)
>> >
>> >
>> > --
>> >
>> > regards,
>> > Laxmikant Upadhyay
>> >
>>
>>
>> -
>> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
>> For additional commands, e-mail: user-h...@cassandra.apache.org
>>
>>
>
> --
>
> regards,
> Laxmikant Upadhyay
>
>

-- 

regards,
Laxmikant Upadhyay


Re: Speed up compaction

2019-06-13 Thread Oleksandr Shulgin
On Thu, Jun 13, 2019 at 11:28 AM Léo FERLIN SUTTON
 wrote:

>
> ## Cassandra configuration :
> 4 concurrent_compactors
> Current compaction throughput: 150 MB/s
> Concurrent reads/write are both set to 128.
>
> I have also temporarily stopped every repair operations.
>
> Any ideas about how I can speed this up ?
>

Hi,

What is the compaction strategy used by this column family?

Do you observe this behavior on one of the nodes only?  Have you tried to
cancel this compaction and see if a new one is started and makes better
progress?  Can you try to restart the affected node?

Regards,
--
Alex


Re: very slow repair

2019-06-13 Thread Oleksandr Shulgin
On Thu, Jun 13, 2019 at 10:36 AM R. T. 
wrote:

>
> Well, actually by running cfstats I can see that the totaldiskspaceused is
> about ~ 1.2 TB per node in the DC1 and ~ 1 TB per node in DC2. DC2 was off
> for a while thats why there is a difference in space.
>
> I am using Cassandra 3.0.6 and
> my stream_throughput_outbound_megabits_per_sec is th4e default setting so
> according to my version is (200 Mbps or 25 MB/s)
>

And the other setting: compaction_throughput_mb_per_sec?  It is also highly
relevant for repair performance, as streamed in files need to be compacted
with the existing files on the nodes.  In our experience change in
compaction throughput limit is almost linearly reflected by the repair run
time.

The default 16 MB/s is too limiting for any production grade setup, I
believe.  We go as high as 90 MB/s on AWS EBS gp2 data volumes.  But don't
take it as a gospel, I'd suggest you start increasing the setting (e.g. by
doubling it) and observe how it affects repair performance (and client
latencies).

Have you tried with "parallel" instead of "DC parallel" mode?  The latter
one is really poorly named and it actually means something else, as neatly
highlighted in this SO answer: https://dba.stackexchange.com/a/175028

Last, but not least: are you using the default number of vnodes, 256?  The
overhead of large number of vnodes (times the number of nodes), can be
quite significant.  We've seen major improvements in repair runtime after
switching from 256 to 16 vnodes on Cassandra version 3.0.

Cheers,
--
Alex


Re: Recover lost node from backup or evict/re-add?

2019-06-13 Thread Oleksandr Shulgin
On Wed, Jun 12, 2019 at 4:02 PM Jeff Jirsa  wrote:

> To avoid violating consistency guarantees, you have to repair the replicas
> while the lost node is down
>

How do you suggest to trigger it?  Potentially replicas of the primary
range for the down node are all over the local DC, so I would go with
triggering a full cluster repair with Cassandra Reaper.  But isn't it going
to fail because of the down node?

It is also documented (I believe) that one should repair the node after it
finishes the "replace address" procedure.  So should one repair before and
after?

--
Alex


Speed up compaction

2019-06-13 Thread Léo FERLIN SUTTON
I am currently noticing very very slow compactions on my cluster and
wondering if there is any way to speed things up.

Right now I have this compaction currently running :

   60c1cfc0-8da7-11e9-bc08-3546c703a280Compaction   keyspace1
> table1 8.77 GB   1.71 TB   bytes  0.50%


The speed is abysmal, it only progresses by "0.24%" every _hour_. At this
rate it is going to take weeks...

When I look at the cpu load and disk load I see :
Never more than 10MB/s read and write combined.
Never more than 6 load average, most of the cpu time is spent idle.

## Hardware configuration :

The node is a n1-highmem-8 (8 vCPUs, 52 GB memory) instance on google cloud
engine.

The disk performances should be around :
15,000.00 random IOPS (read write)
800MB/s Read
400 MB/s Write

## Cassandra configuration :
4 concurrent_compactors
Current compaction throughput: 150 MB/s
Concurrent reads/write are both set to 128.

I have also temporarily stopped every repair operations.

Any ideas about how I can speed this up ?

Regards,

Leo


Re: very slow repair

2019-06-13 Thread R. T.
Hi,

Thank you for your reply,

Well, actually by running cfstats I can see that the totaldiskspaceused is 
about ~ 1.2 TB per node in the DC1 and ~ 1 TB per node in DC2. DC2 was off for 
a while thats why there is a difference in space.

I am using Cassandra 3.0.6 and my stream_throughput_outbound_megabits_per_sec 
is th4e default setting so according to my version is (200 Mbps or 25 MB/s)

Cheers

‐‐‐ Original Message ‐‐‐
On Thursday, June 13, 2019 6:04 AM, Laxmikant Upadhyay 
 wrote:

> Few queries:
> 1. What is the cassandra version ?
> 2. is the size of table 4TB per node ?
> 3. What is the value of compaction_throughput_mb_per_sec and 
> stream_throughput_outbound_megabits_per_sec ?
>
> On Thu, Jun 13, 2019 at 5:06 AM R. T.  wrote:
>
>> Hi,
>>
>> I am trying to run a repair for first time a specific column family in 
>> specific keyspace and it seems that is going super slow.
>>
>> I have 6 nodes cluster with 2 Datacenters (RF 2) and the repair is a non 
>> incremental, DC parallel one. This column family is around 4 TB and it is 
>> written heavily (compared with other CF) so since it is going to take 2 
>> months (according ETA in Reaper) does that mean that when this repair will 
>> finish the entropy will be again high in this CF ?
>>
>> How I can speed up the process ? Is there any way to diagnose bottlenecs?
>>
>> Thank you,
>>
>> W
>
> --
>
> regards,
> Laxmikant Upadhyay

Re: Is my range read query behaving strange ?

2019-06-13 Thread Laxmikant Upadhyay
HI Michael,

Thanks for your reply.
I don't think this issue is related to CASSANDRA-12765
 as in my case the
sstable which has tombstone does not have maxLocalDeletionTime ==
Integer.MAX_VALUE .  I am able to reproduce this issue on 2.1.17 as well.

I am attaching the steps to reproduce on 2.1.17 (with minor change from
previous steps to make sure one request must go to the node which has old
mutation). I have also attached the trace of range read query.

Should I raise a jira for the same ?

On Wed, Jun 12, 2019 at 9:00 AM Michael Shuler 
wrote:

> (dropped dev@ x-post; user@ was correct)
>
> Possibly #12765, fixed in 2.1.17. Wouldn't hurt to update to latest 2.1.21.
>
> https://issues.apache.org/jira/browse/CASSANDRA-12765
> https://github.com/apache/cassandra/blob/cassandra-2.1/CHANGES.txt#L1-L36
>
> Michael
>
> On 6/11/19 9:58 PM, Laxmikant Upadhyay wrote:
> > Does range query ignore purgable tombstone (which crossed grace period)
> > in some cases?
> >
> > On Tue, Jun 11, 2019, 2:56 PM Laxmikant Upadhyay
> > mailto:laxmikant@gmail.com>> wrote:
> >
> > In a 3 node cassandra 2.1.16 cluster where, one node has old
> > mutation and two nodes have evict-able (crossed gc grace period)
> > tombstone produced by TTL. A read range  query with local quorum
> > return the old mutation as result. However expected result should be
> > empty. Next time running the same query results no data as expected.
> > Why this strange behaviour?
> >
> >
> > *Steps to Reproduce :*
> > Create a cassandra-2.1.16  3 node cluster. Disable hinted handoff
> > for each node.
> >
> > #ccm node1 nodetool ring
> > Datacenter: datacenter1
> > ==
> > AddressRackStatus State   LoadOwns
> >   Token
> >
> >3074457345618258602
> > 127.0.0.1  rack1   Up Normal  175.12 KB   100.00%
> >-9223372036854775808
> > 127.0.0.2  rack1   Up   Normal  177.87 KB   100.00%
> >  -3074457345618258603
> > 127.0.0.3  rack1   Up   Normal  175.13 KB   100.00%
> >  3074457345618258602
> >
> > #Connect to cqlsh and set CONISISTENCY LOCAL_QUORUM;
> >
> > cqlsh> CREATE KEYSPACE IF NOT EXISTS test WITH REPLICATION = {
> > 'class' : 'NetworkTopologyStrategy', 'datacenter1' : 3 };
> > cqlsh> CREATE TABLE test.table1 (key text, col text, val
> > text,PRIMARY KEY ((key), col));
> > cqlsh> ALTER TABLE test.table1  with GC_GRACE_SECONDS = 120;
> >
> > cqlsh> INSERT INTO test.table1  (key, col, val) VALUES ('key2',
> > 'abc','xyz');
> >
> > #ccm flush
> >
> > #ccm node3 stop
> >
> > cqlsh> INSERT INTO test.table1  (key, col, val) VALUES ('key2',
> > 'abc','xyz') USING TTL 60;
> >
> > #ccm flush
> >
> > #wait for 3 min so that the tombstone crosses its gc grace period.
> >
> > #ccm node3 start
> >
> > cqlsh> select * from test.table1 where token (key) >
> > 3074457345618258602 and token (key) < -9223372036854775808 ;
> >
> >   key  | col | val
> > --+-+-
> >   key2 | abc | xyz
> >
> > (1 rows)
> >
> > #ccm flush
> > -> Here read repair triggers and the old mutation moves to the one
> > of the node where tombstone is present (not both the node)
> >
> >
> > cqlsh> select * from test.vouchers where token (key) >
> > 3074457345618258602 and token (key) < -9223372036854775808 ;
> >
> >   key | col | val
> > -+-+-
> >
> > (0 rows)
> >
> >
> > --
> >
> > regards,
> > Laxmikant Upadhyay
> >
>
>
> -
> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
> For additional commands, e-mail: user-h...@cassandra.apache.org
>
>

-- 

regards,
Laxmikant Upadhyay
# Create a ccm cluster on version 2.1.17

  # ccm create test1 -v 2.1.17
  
# Populate cluster with 3 nodes.

  # ccm populate -n 3
  
# Disable hinted hand off in all nodes of the cluster 
# Start all the nodes of the cluster

  #ccm start

# Run the nodetool ring on any of the node to find the token ranges.

  # ccm node1 nodetool ring
  
Datacenter: datacenter1
==
AddressRackStatus State   LoadOwnsToken
  
3074457345618258602
127.0.0.1  rack1   Up Normal  46.6 KB 66.67%  
-9223372036854775808
127.0.0.2  rack1   Up Normal  91.86 KB66.67%  
-3074457345618258603
127.0.0.3  rack1   Up Normal  46.65 KB66.67%  
3074457345618258602

# Connect to cqlsh and set CONISISTENCY LOCAL_QUORUM;

  # ccm node1 cqlsh
  cqlsh> CONISISTENCY LOCAL_QUORUM;
  
# Create keyspace with replication factor 3
   
   cqlsh> CREATE KEYSPACE IF NOT EXISTS test WITH REPLICATION = { 'class' :