You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Gianluca Borello <gi...@sysdig.com> on 2016/03/21 20:50:01 UTC

Pending compactions not going down on some nodes of the cluster

Hi,

We added a bunch of new nodes to a cluster (2.1.13) and everything went
fine, except for the number of pending compactions that is staying quite
high on a subset of the new nodes. Over the past 3 days, the pending
compactions have never been less than ~130 on such nodes, with peaks of
~200. On the other nodes, they correctly fluctuate between 0 and ~20, which
has been our norm for a long time.

We are quite paranoid about pending compactions because in the past such
high number caused a lot of data being brought in memory during some reads
and that triggered a chain reaction of full GCs that brought down our
cluster, so we try to monitor them closely.

Some data points that should let the situation speak for itself:

- We use LCS for all our column families

- The cluster is operating absolutely fine and seems healthy, and every
node is handling pretty much the same load in terms of reads and writes.
Also, these nodes with higher pending compactions don't seem in any way
performing worse than the others

- The pending compactions don't go down even when setting the compaction
throughput to unlimited for a very long time

- This is the typical output of compactionstats and tpstats:

$ nodetool compactionstats
pending tasks: 137
   compaction type   keyspace            table     completed         total
   unit   progress
        Compaction     draios   message_data60    6111208394    6939536890
  bytes     88.06%
        Compaction     draios    message_data1   26473390790   37243294809
  bytes     71.08%
Active compaction remaining time :        n/a

$ nodetool tpstats
Pool Name                    Active   Pending      Completed   Blocked  All
time blocked
CounterMutationStage              0         0              0         0
            0
ReadStage                         1         0      111766844         0
            0
RequestResponseStage              0         0      244259493         0
            0
MutationStage                     0         0      163268653         0
            0
ReadRepairStage                   0         0        8933323         0
            0
GossipStage                       0         0         363003         0
            0
CacheCleanupExecutor              0         0              0         0
            0
AntiEntropyStage                  0         0              0         0
            0
MigrationStage                    0         0              2         0
            0
Sampler                           0         0              0         0
            0
ValidationExecutor                0         0              0         0
            0
CommitLogArchiver                 0         0              0         0
            0
MiscStage                         0         0              0         0
            0
MemtableFlushWriter               0         0          32644         0
            0
MemtableReclaimMemory             0         0          32644         0
            0
PendingRangeCalculator            0         0            527         0
            0
MemtablePostFlush                 0         0          36565         0
            0
CompactionExecutor                2        70         108621         0
            0
InternalResponseStage             0         0              0         0
            0
HintedHandoff                     0         0             10         0
            0
Native-Transport-Requests         6         0      188996929         0
        79122

Message type           Dropped
RANGE_SLICE                  0
READ_REPAIR                  0
PAGED_RANGE                  0
BINARY                       0
READ                         0
MUTATION                     0
_TRACE                       0
REQUEST_RESPONSE             0
COUNTER_MUTATION             0

- If I do a nodetool drain on such nodes, and then wait for a while, the
number of pending compactions stays high even if there are no compactions
being executed anymore and the node is completely idle:

$ nodetool compactionstats
pending tasks: 128

- It's also interesting to notice how the compaction in the previous
example is trying to compact ~37 GB, which is essentially the whole size of
the column family message_data1 as reported by cfstats:

$ nodetool cfstats -H draios.message_data1
Keyspace: draios
Read Count: 208168
Read Latency: 2.4791508685292647 ms.
Write Count: 502529
Write Latency: 0.20701542000561163 ms.
Pending Flushes: 0
Table: message_data1
SSTable count: 261
SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
Space used (live): 36.98 GB
Space used (total): 36.98 GB
Space used by snapshots (total): 0 bytes
Off heap memory used (total): 36.21 MB
SSTable Compression Ratio: 0.15461126176169512
Number of keys (estimate): 101025
Memtable cell count: 229344
Memtable data size: 82.4 MB
Memtable off heap memory used: 0 bytes
Memtable switch count: 83
Local read count: 208225
Local read latency: 2.479 ms
Local write count: 502581
Local write latency: 0.208 ms
Pending flushes: 0
Bloom filter false positives: 11497
Bloom filter false ratio: 0.04307
Bloom filter space used: 94.97 KB
Bloom filter off heap memory used: 92.93 KB
Index summary off heap memory used: 57.88 KB
Compression metadata off heap memory used: 36.06 MB
Compacted partition minimum bytes: 447 bytes
Compacted partition maximum bytes: 34.48 MB
Compacted partition mean bytes: 1.51 MB
Average live cells per slice (last five minutes): 26.269698643294515
Maximum live cells per slice (last five minutes): 100.0
Average tombstones per slice (last five minutes): 0.0
Maximum tombstones per slice (last five minutes): 0.0

- There are no warnings or errors in the log, even after a clean restart

- Restarting the node doesn't seem to have any effect on the number of
pending compactions

Any help would be very appreciated.

Thank you for reading

Re: Pending compactions not going down on some nodes of the cluster

Posted by Gianluca Borello <gi...@sysdig.com>.
On Mon, Mar 21, 2016 at 2:15 PM, Alain RODRIGUEZ <ar...@gmail.com> wrote:

>
> What hardware do you use? Can you see it running at the limits (CPU /
> disks IO)? Is there any error on system logs, are disks doing fine?
>
>
Nodes are c3.2xlarge instances on AWS. The nodes are relatively idle, and,
as said in the original email, the other nodes are handling the same load
and are doing just fine (no warnings or errors in the system and cassandra
logs). I mean, even this node is doing very fine, it seems as if there is
some internal bug in the way pending tasks are counted.


> How is configured the concurrent_compactors setting? It looks like you are
> using 2 from 'nodetool compactionstats' output. Could you increase that
> or is it the number of cores of the machine?
>
>
It's 2. I can certainly increase it, but, as I proved earlier, it doesn't
do anything, because if I do a "nodetool drain" and wait a while, the
compactions completely stop (and the node goes totally idle), despite the
pending compactions remaining high, so this really doesn't feel like a
problem of Cassandra not being able to keep up with the compactions.

Re: Pending compactions not going down on some nodes of the cluster

Posted by Alain RODRIGUEZ <ar...@gmail.com>.
Hi, thanks for the detailed information, it is useful.

SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]


Looks like compaction is not doing so hot indeed.

What hardware do you use? Can you see it running at the limits (CPU / disks
IO)? Is there any error on system logs, are disks doing fine?

How is configured the concurrent_compactors setting? It looks like you are
using 2 from 'nodetool compactionstats' output. Could you increase that or
is it the number of cores of the machine?

What size do you use for LCS sstables? Using a small size might produce too
many sstables.

Have those nodes more sstables than the other ones?

Are compactions progressing or getting stuck at some point?

It's also interesting to notice how the compaction in the previous example
> is trying to compact ~37 GB, which is essentially the whole size of the
> column family message_data1 as reported by cfstats


Weird indeed, using LCS. Not a clue about that but a small tip: you can use
'nodetool compactionstats *-H*' to have values in human readable format
since 2.1 :-).

Good luck,
-----------------------
Alain Rodriguez - alain@thelastpickle.com
France

The Last Pickle - Apache Cassandra Consulting
http://www.thelastpickle.com

2016-03-21 20:50 GMT+01:00 Gianluca Borello <gi...@sysdig.com>:

> Hi,
>
> We added a bunch of new nodes to a cluster (2.1.13) and everything went
> fine, except for the number of pending compactions that is staying quite
> high on a subset of the new nodes. Over the past 3 days, the pending
> compactions have never been less than ~130 on such nodes, with peaks of
> ~200. On the other nodes, they correctly fluctuate between 0 and ~20, which
> has been our norm for a long time.
>
> We are quite paranoid about pending compactions because in the past such
> high number caused a lot of data being brought in memory during some reads
> and that triggered a chain reaction of full GCs that brought down our
> cluster, so we try to monitor them closely.
>
> Some data points that should let the situation speak for itself:
>
> - We use LCS for all our column families
>
> - The cluster is operating absolutely fine and seems healthy, and every
> node is handling pretty much the same load in terms of reads and writes.
> Also, these nodes with higher pending compactions don't seem in any way
> performing worse than the others
>
> - The pending compactions don't go down even when setting the compaction
> throughput to unlimited for a very long time
>
> - This is the typical output of compactionstats and tpstats:
>
> $ nodetool compactionstats
> pending tasks: 137
>    compaction type   keyspace            table     completed         total
>    unit   progress
>         Compaction     draios   message_data60    6111208394    6939536890
>   bytes     88.06%
>         Compaction     draios    message_data1   26473390790   37243294809
>   bytes     71.08%
> Active compaction remaining time :        n/a
>
> $ nodetool tpstats
> Pool Name                    Active   Pending      Completed   Blocked
>  All time blocked
> CounterMutationStage              0         0              0         0
>             0
> ReadStage                         1         0      111766844         0
>             0
> RequestResponseStage              0         0      244259493         0
>             0
> MutationStage                     0         0      163268653         0
>             0
> ReadRepairStage                   0         0        8933323         0
>             0
> GossipStage                       0         0         363003         0
>             0
> CacheCleanupExecutor              0         0              0         0
>             0
> AntiEntropyStage                  0         0              0         0
>             0
> MigrationStage                    0         0              2         0
>             0
> Sampler                           0         0              0         0
>             0
> ValidationExecutor                0         0              0         0
>             0
> CommitLogArchiver                 0         0              0         0
>             0
> MiscStage                         0         0              0         0
>             0
> MemtableFlushWriter               0         0          32644         0
>             0
> MemtableReclaimMemory             0         0          32644         0
>             0
> PendingRangeCalculator            0         0            527         0
>             0
> MemtablePostFlush                 0         0          36565         0
>             0
> CompactionExecutor                2        70         108621         0
>             0
> InternalResponseStage             0         0              0         0
>             0
> HintedHandoff                     0         0             10         0
>             0
> Native-Transport-Requests         6         0      188996929         0
>         79122
>
> Message type           Dropped
> RANGE_SLICE                  0
> READ_REPAIR                  0
> PAGED_RANGE                  0
> BINARY                       0
> READ                         0
> MUTATION                     0
> _TRACE                       0
> REQUEST_RESPONSE             0
> COUNTER_MUTATION             0
>
> - If I do a nodetool drain on such nodes, and then wait for a while, the
> number of pending compactions stays high even if there are no compactions
> being executed anymore and the node is completely idle:
>
> $ nodetool compactionstats
> pending tasks: 128
>
> - It's also interesting to notice how the compaction in the previous
> example is trying to compact ~37 GB, which is essentially the whole size of
> the column family message_data1 as reported by cfstats:
>
> $ nodetool cfstats -H draios.message_data1
> Keyspace: draios
> Read Count: 208168
> Read Latency: 2.4791508685292647 ms.
> Write Count: 502529
> Write Latency: 0.20701542000561163 ms.
> Pending Flushes: 0
> Table: message_data1
> SSTable count: 261
> SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
> Space used (live): 36.98 GB
> Space used (total): 36.98 GB
> Space used by snapshots (total): 0 bytes
> Off heap memory used (total): 36.21 MB
> SSTable Compression Ratio: 0.15461126176169512
> Number of keys (estimate): 101025
> Memtable cell count: 229344
> Memtable data size: 82.4 MB
> Memtable off heap memory used: 0 bytes
> Memtable switch count: 83
> Local read count: 208225
> Local read latency: 2.479 ms
> Local write count: 502581
> Local write latency: 0.208 ms
> Pending flushes: 0
> Bloom filter false positives: 11497
> Bloom filter false ratio: 0.04307
> Bloom filter space used: 94.97 KB
> Bloom filter off heap memory used: 92.93 KB
> Index summary off heap memory used: 57.88 KB
> Compression metadata off heap memory used: 36.06 MB
> Compacted partition minimum bytes: 447 bytes
> Compacted partition maximum bytes: 34.48 MB
> Compacted partition mean bytes: 1.51 MB
> Average live cells per slice (last five minutes): 26.269698643294515
> Maximum live cells per slice (last five minutes): 100.0
> Average tombstones per slice (last five minutes): 0.0
> Maximum tombstones per slice (last five minutes): 0.0
>
> - There are no warnings or errors in the log, even after a clean restart
>
> - Restarting the node doesn't seem to have any effect on the number of
> pending compactions
>
> Any help would be very appreciated.
>
> Thank you for reading
>
>

Re: Pending compactions not going down on some nodes of the cluster

Posted by Gianluca Borello <gi...@sysdig.com>.
On Mon, Mar 21, 2016 at 12:50 PM, Gianluca Borello <gi...@sysdig.com>
wrote:

>
> - It's also interesting to notice how the compaction in the previous
> example is trying to compact ~37 GB, which is essentially the whole size of
> the column family message_data1 as reported by cfstats:
>

Also related to this point, now I'm seeing something even more odd: some
compactions are way bigger than the size of the column family itself, such
as:

$ nodetool compactionstats -H
pending tasks: 110
   compaction type   keyspace           table   completed       total
 unit   progress
        Compaction     draios   message_data1    26.28 GB   118.73 GB
bytes     22.13%
Active compaction remaining time :   0h04m30s

It says the total is 118.73 GB, but that column family never got anywhere
close to that size, it has always stayed around ~30GB:

$ du -hs /raid0/cassandra/data/draios/message_data1-*
35G
/raid0/cassandra/data/draios/message_data1-ad87f550ea2b11e5a528dde586fa678e

Re: Pending compactions not going down on some nodes of the cluster

Posted by Alain RODRIGUEZ <ar...@gmail.com>.
Hi,

Any improvement on this?

2 ideas coming to my mind:

Yes, we are storing timeseries-like binary blobs where data is heavily
> TTLed (essentially the entire column family is incrementally refreshed with
> completely new data every few days)


This looks to me like a good fit for TWCS (https://github.com/jeffjirsa/twcs),
Jeff would know that better than I ;-), or DTCS developed by Marcus for
Spotify back then (
https://labs.spotify.com/2014/12/18/date-tiered-compaction/). This way you
would alleviate substancially compactions need as all the expired SSTables
would be removed with no compaction. The second one is mainstream it is
present in Cassandra, TWCS is not, but has been built to work around some
DTCS issues and is heavy usage and maintained by Jeff for now.

 I drain the node, the compactions completely stop after a few minutes
> (like they would normally do on another "healthy" node), it's just the
> "pending tasks" counter that stays high, and messes up my metrics in
> OpsCenter


Maybe is there a compaction stuck there. It happened to me in the past.
Checking sstables considered in remaining compaction after you drained can
be helpful to troubleshoot.

Run: "echo 'SELECT * FROM system.compactions_in_progress;' | cqlsh" and try
to find that you indeed have compactions pending (there were some issue
with the pending compaction metric recently and from the 'inputs' table
look for some sstable(s) that is present in all the operation (on multiple
sstables being used in many compactions processes. When I had the issue of
a sstable blocking the compactions, it was very obvious.

You can also try: " echo 'TRUNCATE system.compactions_in_progress;' |
cqlsh" and restart the node, see what happen.

If you spot some sstable messing in some node, you can try removing the
sstable and repairing the table (one node at the time).

C*heers,
-----------------------
Alain Rodriguez - alain@thelastpickle.com
France

The Last Pickle - Apache Cassandra Consulting
http://www.thelastpickle.com

2016-03-22 6:02 GMT+01:00 Gianluca Borello <gi...@sysdig.com>:

> Thank you for your reply. To address your points:
>
> - We are not running repairs
>
> - Yes, we are storing timeseries-like binary blobs where data is heavily
> TTLed (essentially the entire column family is incrementally refreshed with
> completely new data every few days)
>
> - I have tried with increasing compactors and even putting the compaction
> throughput to unlimited, and nothing changes. Again, the key here to
> remember is that if I drain the node, the compactions completely stop after
> a few minutes (like they would normally do on another "healthy" node), it's
> just the "pending tasks" counter that stays high, and messes up my metrics
> in OpsCenter
>
> - As another data point: even if I increase the number of resources
> allocated to compactions, I can pretty much measure that the disk I/O
> generated by Cassandra is essentially the same as the other nodes that have
> no pending compactions. In other words, it really seems like that number of
> estimated pending compactions is somehow bogus
>
> Thanks
>
> On Mon, Mar 21, 2016 at 9:45 PM, Fabrice Facorat <
> fabrice.facorat@gmail.com> wrote:
>
>> Are you running repairs ?
>>
>> You may try:
>> - increase concurrentçcompaction to 8 (max in 2.1.x)
>> - increase compaction_throupghput to more than 16MB/s (48 may be a good
>> start)
>>
>>
>> What kind of data are you storing in theses tables ? timeseries ?
>>
>>
>>
>> 2016-03-21 23:37 GMT+01:00 Gianluca Borello <gi...@sysdig.com>:
>> > Thank you for your reply, definitely appreciate the tip on the
>> compressed
>> > size.
>> >
>> > I understand your point, in fact whenever we bootstrap a new node we
>> see a
>> > huge number of pending compactions (in the order of thousands), and they
>> > usually decrease steadily until they reach 0 in just a few hours. With
>> this
>> > node, however, we are way beyond that point, it has been 3 days since
>> the
>> > number of pending compaction started fluctuating around ~150 without any
>> > sign of going down (I can see from Opscenter it's almost a straight line
>> > starting a few hours after the bootstrap). In particular, to reply to
>> your
>> > point:
>> >
>> > - The number of sstables for this CF on this node is around 250, which
>> is in
>> > the same range of all the other nodes in the cluster (I counted the
>> number
>> > on each one of them, and every node is in the 200-400 range)
>> >
>> > - This theory doesn't seem to explain why, when doing "nodetool drain",
>> the
>> > compactions completely stop after a few minutes and I get something
>> such as:
>> >
>> > $ nodetool compactionstats
>> > pending tasks: 128
>> >
>> > So no compactions being executed (since there is no more write
>> activity),
>> > but the pending number is still high.
>> >
>> > Thanks again
>> >
>> >
>> > On Mon, Mar 21, 2016 at 3:19 PM, Jeff Jirsa <jeff.jirsa@crowdstrike.com
>> >
>> > wrote:
>> >>
>> >> > We added a bunch of new nodes to a cluster (2.1.13) and everything
>> went
>> >> > fine, except for the number of pending compactions that is staying
>> quite
>> >> > high on a subset of the new nodes. Over the past 3 days, the pending
>> >> > compactions have never been less than ~130 on such nodes, with peaks
>> of
>> >> > ~200.
>> >>
>> >> When you bootstrap with Vnodes, you end up with thousands (or tens of
>> >> thousands) of sstables – with 256 Vnodes (default) * 20 sstables per
>> node,
>> >> your resulting node will have 5k sstables. It takes quite a while for
>> >> compaction to chew through that. If you added a bunch of nodes in
>> sequence,
>> >> you’d have 5k on the first node, then potentailly 10k on the next, and
>> could
>> >> potentially keep increasing as you start streaming from nodes that
>> have way
>> >> too many sstables.  This is one of the reasons that many people who
>> have to
>> >> grow their clusters frequently try not to use vnodes.
>> >>
>> >> From your other email:
>> >>
>> >> > Also related to this point, now I'm seeing something even more odd:
>> some
>> >> > compactions are way bigger than the size of the column family
>> itself, such
>> >> > as:
>> >>
>> >> The size reported by compactionstats is the uncompressed size – if
>> you’re
>> >> using compression, it’s perfectly reasonable for 30G of data to show
>> up as
>> >> 118G of data during compaction.
>> >>
>> >> - Jeff
>> >>
>> >> From: Gianluca Borello
>> >> Reply-To: "user@cassandra.apache.org"
>> >> Date: Monday, March 21, 2016 at 12:50 PM
>> >> To: "user@cassandra.apache.org"
>> >> Subject: Pending compactions not going down on some nodes of the
>> cluster
>> >>
>> >> Hi,
>> >>
>> >> We added a bunch of new nodes to a cluster (2.1.13) and everything went
>> >> fine, except for the number of pending compactions that is staying
>> quite
>> >> high on a subset of the new nodes. Over the past 3 days, the pending
>> >> compactions have never been less than ~130 on such nodes, with peaks of
>> >> ~200. On the other nodes, they correctly fluctuate between 0 and ~20,
>> which
>> >> has been our norm for a long time.
>> >>
>> >> We are quite paranoid about pending compactions because in the past
>> such
>> >> high number caused a lot of data being brought in memory during some
>> reads
>> >> and that triggered a chain reaction of full GCs that brought down our
>> >> cluster, so we try to monitor them closely.
>> >>
>> >> Some data points that should let the situation speak for itself:
>> >>
>> >> - We use LCS for all our column families
>> >>
>> >> - The cluster is operating absolutely fine and seems healthy, and every
>> >> node is handling pretty much the same load in terms of reads and
>> writes.
>> >> Also, these nodes with higher pending compactions don't seem in any way
>> >> performing worse than the others
>> >>
>> >> - The pending compactions don't go down even when setting the
>> compaction
>> >> throughput to unlimited for a very long time
>> >>
>> >> - This is the typical output of compactionstats and tpstats:
>> >>
>> >> $ nodetool compactionstats
>> >> pending tasks: 137
>> >>    compaction type   keyspace            table     completed
>>  total
>> >> unit   progress
>> >>         Compaction     draios   message_data60    6111208394
>> 6939536890
>> >> bytes     88.06%
>> >>         Compaction     draios    message_data1   26473390790
>>  37243294809
>> >> bytes     71.08%
>> >> Active compaction remaining time :        n/a
>> >>
>> >> $ nodetool tpstats
>> >> Pool Name                    Active   Pending      Completed   Blocked
>> >> All time blocked
>> >> CounterMutationStage              0         0              0         0
>> >> 0
>> >> ReadStage                         1         0      111766844         0
>> >> 0
>> >> RequestResponseStage              0         0      244259493         0
>> >> 0
>> >> MutationStage                     0         0      163268653         0
>> >> 0
>> >> ReadRepairStage                   0         0        8933323         0
>> >> 0
>> >> GossipStage                       0         0         363003         0
>> >> 0
>> >> CacheCleanupExecutor              0         0              0         0
>> >> 0
>> >> AntiEntropyStage                  0         0              0         0
>> >> 0
>> >> MigrationStage                    0         0              2         0
>> >> 0
>> >> Sampler                           0         0              0         0
>> >> 0
>> >> ValidationExecutor                0         0              0         0
>> >> 0
>> >> CommitLogArchiver                 0         0              0         0
>> >> 0
>> >> MiscStage                         0         0              0         0
>> >> 0
>> >> MemtableFlushWriter               0         0          32644         0
>> >> 0
>> >> MemtableReclaimMemory             0         0          32644         0
>> >> 0
>> >> PendingRangeCalculator            0         0            527         0
>> >> 0
>> >> MemtablePostFlush                 0         0          36565         0
>> >> 0
>> >> CompactionExecutor                2        70         108621         0
>> >> 0
>> >> InternalResponseStage             0         0              0         0
>> >> 0
>> >> HintedHandoff                     0         0             10         0
>> >> 0
>> >> Native-Transport-Requests         6         0      188996929         0
>> >> 79122
>> >>
>> >> Message type           Dropped
>> >> RANGE_SLICE                  0
>> >> READ_REPAIR                  0
>> >> PAGED_RANGE                  0
>> >> BINARY                       0
>> >> READ                         0
>> >> MUTATION                     0
>> >> _TRACE                       0
>> >> REQUEST_RESPONSE             0
>> >> COUNTER_MUTATION             0
>> >>
>> >> - If I do a nodetool drain on such nodes, and then wait for a while,
>> the
>> >> number of pending compactions stays high even if there are no
>> compactions
>> >> being executed anymore and the node is completely idle:
>> >>
>> >> $ nodetool compactionstats
>> >> pending tasks: 128
>> >>
>> >> - It's also interesting to notice how the compaction in the previous
>> >> example is trying to compact ~37 GB, which is essentially the whole
>> size of
>> >> the column family message_data1 as reported by cfstats:
>> >>
>> >> $ nodetool cfstats -H draios.message_data1
>> >> Keyspace: draios
>> >> Read Count: 208168
>> >> Read Latency: 2.4791508685292647 ms.
>> >> Write Count: 502529
>> >> Write Latency: 0.20701542000561163 ms.
>> >> Pending Flushes: 0
>> >> Table: message_data1
>> >> SSTable count: 261
>> >> SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
>> >> Space used (live): 36.98 GB
>> >> Space used (total): 36.98 GB
>> >> Space used by snapshots (total): 0 bytes
>> >> Off heap memory used (total): 36.21 MB
>> >> SSTable Compression Ratio: 0.15461126176169512
>> >> Number of keys (estimate): 101025
>> >> Memtable cell count: 229344
>> >> Memtable data size: 82.4 MB
>> >> Memtable off heap memory used: 0 bytes
>> >> Memtable switch count: 83
>> >> Local read count: 208225
>> >> Local read latency: 2.479 ms
>> >> Local write count: 502581
>> >> Local write latency: 0.208 ms
>> >> Pending flushes: 0
>> >> Bloom filter false positives: 11497
>> >> Bloom filter false ratio: 0.04307
>> >> Bloom filter space used: 94.97 KB
>> >> Bloom filter off heap memory used: 92.93 KB
>> >> Index summary off heap memory used: 57.88 KB
>> >> Compression metadata off heap memory used: 36.06 MB
>> >> Compacted partition minimum bytes: 447 bytes
>> >> Compacted partition maximum bytes: 34.48 MB
>> >> Compacted partition mean bytes: 1.51 MB
>> >> Average live cells per slice (last five minutes): 26.269698643294515
>> >> Maximum live cells per slice (last five minutes): 100.0
>> >> Average tombstones per slice (last five minutes): 0.0
>> >> Maximum tombstones per slice (last five minutes): 0.0
>> >>
>> >> - There are no warnings or errors in the log, even after a clean
>> restart
>> >>
>> >> - Restarting the node doesn't seem to have any effect on the number of
>> >> pending compactions
>> >>
>> >> Any help would be very appreciated.
>> >>
>> >> Thank you for reading
>> >>
>> >
>>
>>
>>
>> --
>> Close the World, Open the Net
>> http://www.linux-wizard.net
>>
>
>

Re: Pending compactions not going down on some nodes of the cluster

Posted by Gianluca Borello <gi...@sysdig.com>.
Thank you for your reply. To address your points:

- We are not running repairs

- Yes, we are storing timeseries-like binary blobs where data is heavily
TTLed (essentially the entire column family is incrementally refreshed with
completely new data every few days)

- I have tried with increasing compactors and even putting the compaction
throughput to unlimited, and nothing changes. Again, the key here to
remember is that if I drain the node, the compactions completely stop after
a few minutes (like they would normally do on another "healthy" node), it's
just the "pending tasks" counter that stays high, and messes up my metrics
in OpsCenter

- As another data point: even if I increase the number of resources
allocated to compactions, I can pretty much measure that the disk I/O
generated by Cassandra is essentially the same as the other nodes that have
no pending compactions. In other words, it really seems like that number of
estimated pending compactions is somehow bogus

Thanks

On Mon, Mar 21, 2016 at 9:45 PM, Fabrice Facorat <fa...@gmail.com>
wrote:

> Are you running repairs ?
>
> You may try:
> - increase concurrentçcompaction to 8 (max in 2.1.x)
> - increase compaction_throupghput to more than 16MB/s (48 may be a good
> start)
>
>
> What kind of data are you storing in theses tables ? timeseries ?
>
>
>
> 2016-03-21 23:37 GMT+01:00 Gianluca Borello <gi...@sysdig.com>:
> > Thank you for your reply, definitely appreciate the tip on the compressed
> > size.
> >
> > I understand your point, in fact whenever we bootstrap a new node we see
> a
> > huge number of pending compactions (in the order of thousands), and they
> > usually decrease steadily until they reach 0 in just a few hours. With
> this
> > node, however, we are way beyond that point, it has been 3 days since the
> > number of pending compaction started fluctuating around ~150 without any
> > sign of going down (I can see from Opscenter it's almost a straight line
> > starting a few hours after the bootstrap). In particular, to reply to
> your
> > point:
> >
> > - The number of sstables for this CF on this node is around 250, which
> is in
> > the same range of all the other nodes in the cluster (I counted the
> number
> > on each one of them, and every node is in the 200-400 range)
> >
> > - This theory doesn't seem to explain why, when doing "nodetool drain",
> the
> > compactions completely stop after a few minutes and I get something such
> as:
> >
> > $ nodetool compactionstats
> > pending tasks: 128
> >
> > So no compactions being executed (since there is no more write activity),
> > but the pending number is still high.
> >
> > Thanks again
> >
> >
> > On Mon, Mar 21, 2016 at 3:19 PM, Jeff Jirsa <je...@crowdstrike.com>
> > wrote:
> >>
> >> > We added a bunch of new nodes to a cluster (2.1.13) and everything
> went
> >> > fine, except for the number of pending compactions that is staying
> quite
> >> > high on a subset of the new nodes. Over the past 3 days, the pending
> >> > compactions have never been less than ~130 on such nodes, with peaks
> of
> >> > ~200.
> >>
> >> When you bootstrap with Vnodes, you end up with thousands (or tens of
> >> thousands) of sstables – with 256 Vnodes (default) * 20 sstables per
> node,
> >> your resulting node will have 5k sstables. It takes quite a while for
> >> compaction to chew through that. If you added a bunch of nodes in
> sequence,
> >> you’d have 5k on the first node, then potentailly 10k on the next, and
> could
> >> potentially keep increasing as you start streaming from nodes that have
> way
> >> too many sstables.  This is one of the reasons that many people who
> have to
> >> grow their clusters frequently try not to use vnodes.
> >>
> >> From your other email:
> >>
> >> > Also related to this point, now I'm seeing something even more odd:
> some
> >> > compactions are way bigger than the size of the column family itself,
> such
> >> > as:
> >>
> >> The size reported by compactionstats is the uncompressed size – if
> you’re
> >> using compression, it’s perfectly reasonable for 30G of data to show up
> as
> >> 118G of data during compaction.
> >>
> >> - Jeff
> >>
> >> From: Gianluca Borello
> >> Reply-To: "user@cassandra.apache.org"
> >> Date: Monday, March 21, 2016 at 12:50 PM
> >> To: "user@cassandra.apache.org"
> >> Subject: Pending compactions not going down on some nodes of the cluster
> >>
> >> Hi,
> >>
> >> We added a bunch of new nodes to a cluster (2.1.13) and everything went
> >> fine, except for the number of pending compactions that is staying quite
> >> high on a subset of the new nodes. Over the past 3 days, the pending
> >> compactions have never been less than ~130 on such nodes, with peaks of
> >> ~200. On the other nodes, they correctly fluctuate between 0 and ~20,
> which
> >> has been our norm for a long time.
> >>
> >> We are quite paranoid about pending compactions because in the past such
> >> high number caused a lot of data being brought in memory during some
> reads
> >> and that triggered a chain reaction of full GCs that brought down our
> >> cluster, so we try to monitor them closely.
> >>
> >> Some data points that should let the situation speak for itself:
> >>
> >> - We use LCS for all our column families
> >>
> >> - The cluster is operating absolutely fine and seems healthy, and every
> >> node is handling pretty much the same load in terms of reads and writes.
> >> Also, these nodes with higher pending compactions don't seem in any way
> >> performing worse than the others
> >>
> >> - The pending compactions don't go down even when setting the compaction
> >> throughput to unlimited for a very long time
> >>
> >> - This is the typical output of compactionstats and tpstats:
> >>
> >> $ nodetool compactionstats
> >> pending tasks: 137
> >>    compaction type   keyspace            table     completed
>  total
> >> unit   progress
> >>         Compaction     draios   message_data60    6111208394
> 6939536890
> >> bytes     88.06%
> >>         Compaction     draios    message_data1   26473390790
>  37243294809
> >> bytes     71.08%
> >> Active compaction remaining time :        n/a
> >>
> >> $ nodetool tpstats
> >> Pool Name                    Active   Pending      Completed   Blocked
> >> All time blocked
> >> CounterMutationStage              0         0              0         0
> >> 0
> >> ReadStage                         1         0      111766844         0
> >> 0
> >> RequestResponseStage              0         0      244259493         0
> >> 0
> >> MutationStage                     0         0      163268653         0
> >> 0
> >> ReadRepairStage                   0         0        8933323         0
> >> 0
> >> GossipStage                       0         0         363003         0
> >> 0
> >> CacheCleanupExecutor              0         0              0         0
> >> 0
> >> AntiEntropyStage                  0         0              0         0
> >> 0
> >> MigrationStage                    0         0              2         0
> >> 0
> >> Sampler                           0         0              0         0
> >> 0
> >> ValidationExecutor                0         0              0         0
> >> 0
> >> CommitLogArchiver                 0         0              0         0
> >> 0
> >> MiscStage                         0         0              0         0
> >> 0
> >> MemtableFlushWriter               0         0          32644         0
> >> 0
> >> MemtableReclaimMemory             0         0          32644         0
> >> 0
> >> PendingRangeCalculator            0         0            527         0
> >> 0
> >> MemtablePostFlush                 0         0          36565         0
> >> 0
> >> CompactionExecutor                2        70         108621         0
> >> 0
> >> InternalResponseStage             0         0              0         0
> >> 0
> >> HintedHandoff                     0         0             10         0
> >> 0
> >> Native-Transport-Requests         6         0      188996929         0
> >> 79122
> >>
> >> Message type           Dropped
> >> RANGE_SLICE                  0
> >> READ_REPAIR                  0
> >> PAGED_RANGE                  0
> >> BINARY                       0
> >> READ                         0
> >> MUTATION                     0
> >> _TRACE                       0
> >> REQUEST_RESPONSE             0
> >> COUNTER_MUTATION             0
> >>
> >> - If I do a nodetool drain on such nodes, and then wait for a while, the
> >> number of pending compactions stays high even if there are no
> compactions
> >> being executed anymore and the node is completely idle:
> >>
> >> $ nodetool compactionstats
> >> pending tasks: 128
> >>
> >> - It's also interesting to notice how the compaction in the previous
> >> example is trying to compact ~37 GB, which is essentially the whole
> size of
> >> the column family message_data1 as reported by cfstats:
> >>
> >> $ nodetool cfstats -H draios.message_data1
> >> Keyspace: draios
> >> Read Count: 208168
> >> Read Latency: 2.4791508685292647 ms.
> >> Write Count: 502529
> >> Write Latency: 0.20701542000561163 ms.
> >> Pending Flushes: 0
> >> Table: message_data1
> >> SSTable count: 261
> >> SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
> >> Space used (live): 36.98 GB
> >> Space used (total): 36.98 GB
> >> Space used by snapshots (total): 0 bytes
> >> Off heap memory used (total): 36.21 MB
> >> SSTable Compression Ratio: 0.15461126176169512
> >> Number of keys (estimate): 101025
> >> Memtable cell count: 229344
> >> Memtable data size: 82.4 MB
> >> Memtable off heap memory used: 0 bytes
> >> Memtable switch count: 83
> >> Local read count: 208225
> >> Local read latency: 2.479 ms
> >> Local write count: 502581
> >> Local write latency: 0.208 ms
> >> Pending flushes: 0
> >> Bloom filter false positives: 11497
> >> Bloom filter false ratio: 0.04307
> >> Bloom filter space used: 94.97 KB
> >> Bloom filter off heap memory used: 92.93 KB
> >> Index summary off heap memory used: 57.88 KB
> >> Compression metadata off heap memory used: 36.06 MB
> >> Compacted partition minimum bytes: 447 bytes
> >> Compacted partition maximum bytes: 34.48 MB
> >> Compacted partition mean bytes: 1.51 MB
> >> Average live cells per slice (last five minutes): 26.269698643294515
> >> Maximum live cells per slice (last five minutes): 100.0
> >> Average tombstones per slice (last five minutes): 0.0
> >> Maximum tombstones per slice (last five minutes): 0.0
> >>
> >> - There are no warnings or errors in the log, even after a clean restart
> >>
> >> - Restarting the node doesn't seem to have any effect on the number of
> >> pending compactions
> >>
> >> Any help would be very appreciated.
> >>
> >> Thank you for reading
> >>
> >
>
>
>
> --
> Close the World, Open the Net
> http://www.linux-wizard.net
>

Re: Pending compactions not going down on some nodes of the cluster

Posted by Fabrice Facorat <fa...@gmail.com>.
Are you running repairs ?

You may try:
- increase concurrentçcompaction to 8 (max in 2.1.x)
- increase compaction_throupghput to more than 16MB/s (48 may be a good start)


What kind of data are you storing in theses tables ? timeseries ?



2016-03-21 23:37 GMT+01:00 Gianluca Borello <gi...@sysdig.com>:
> Thank you for your reply, definitely appreciate the tip on the compressed
> size.
>
> I understand your point, in fact whenever we bootstrap a new node we see a
> huge number of pending compactions (in the order of thousands), and they
> usually decrease steadily until they reach 0 in just a few hours. With this
> node, however, we are way beyond that point, it has been 3 days since the
> number of pending compaction started fluctuating around ~150 without any
> sign of going down (I can see from Opscenter it's almost a straight line
> starting a few hours after the bootstrap). In particular, to reply to your
> point:
>
> - The number of sstables for this CF on this node is around 250, which is in
> the same range of all the other nodes in the cluster (I counted the number
> on each one of them, and every node is in the 200-400 range)
>
> - This theory doesn't seem to explain why, when doing "nodetool drain", the
> compactions completely stop after a few minutes and I get something such as:
>
> $ nodetool compactionstats
> pending tasks: 128
>
> So no compactions being executed (since there is no more write activity),
> but the pending number is still high.
>
> Thanks again
>
>
> On Mon, Mar 21, 2016 at 3:19 PM, Jeff Jirsa <je...@crowdstrike.com>
> wrote:
>>
>> > We added a bunch of new nodes to a cluster (2.1.13) and everything went
>> > fine, except for the number of pending compactions that is staying quite
>> > high on a subset of the new nodes. Over the past 3 days, the pending
>> > compactions have never been less than ~130 on such nodes, with peaks of
>> > ~200.
>>
>> When you bootstrap with Vnodes, you end up with thousands (or tens of
>> thousands) of sstables – with 256 Vnodes (default) * 20 sstables per node,
>> your resulting node will have 5k sstables. It takes quite a while for
>> compaction to chew through that. If you added a bunch of nodes in sequence,
>> you’d have 5k on the first node, then potentailly 10k on the next, and could
>> potentially keep increasing as you start streaming from nodes that have way
>> too many sstables.  This is one of the reasons that many people who have to
>> grow their clusters frequently try not to use vnodes.
>>
>> From your other email:
>>
>> > Also related to this point, now I'm seeing something even more odd: some
>> > compactions are way bigger than the size of the column family itself, such
>> > as:
>>
>> The size reported by compactionstats is the uncompressed size – if you’re
>> using compression, it’s perfectly reasonable for 30G of data to show up as
>> 118G of data during compaction.
>>
>> - Jeff
>>
>> From: Gianluca Borello
>> Reply-To: "user@cassandra.apache.org"
>> Date: Monday, March 21, 2016 at 12:50 PM
>> To: "user@cassandra.apache.org"
>> Subject: Pending compactions not going down on some nodes of the cluster
>>
>> Hi,
>>
>> We added a bunch of new nodes to a cluster (2.1.13) and everything went
>> fine, except for the number of pending compactions that is staying quite
>> high on a subset of the new nodes. Over the past 3 days, the pending
>> compactions have never been less than ~130 on such nodes, with peaks of
>> ~200. On the other nodes, they correctly fluctuate between 0 and ~20, which
>> has been our norm for a long time.
>>
>> We are quite paranoid about pending compactions because in the past such
>> high number caused a lot of data being brought in memory during some reads
>> and that triggered a chain reaction of full GCs that brought down our
>> cluster, so we try to monitor them closely.
>>
>> Some data points that should let the situation speak for itself:
>>
>> - We use LCS for all our column families
>>
>> - The cluster is operating absolutely fine and seems healthy, and every
>> node is handling pretty much the same load in terms of reads and writes.
>> Also, these nodes with higher pending compactions don't seem in any way
>> performing worse than the others
>>
>> - The pending compactions don't go down even when setting the compaction
>> throughput to unlimited for a very long time
>>
>> - This is the typical output of compactionstats and tpstats:
>>
>> $ nodetool compactionstats
>> pending tasks: 137
>>    compaction type   keyspace            table     completed         total
>> unit   progress
>>         Compaction     draios   message_data60    6111208394    6939536890
>> bytes     88.06%
>>         Compaction     draios    message_data1   26473390790   37243294809
>> bytes     71.08%
>> Active compaction remaining time :        n/a
>>
>> $ nodetool tpstats
>> Pool Name                    Active   Pending      Completed   Blocked
>> All time blocked
>> CounterMutationStage              0         0              0         0
>> 0
>> ReadStage                         1         0      111766844         0
>> 0
>> RequestResponseStage              0         0      244259493         0
>> 0
>> MutationStage                     0         0      163268653         0
>> 0
>> ReadRepairStage                   0         0        8933323         0
>> 0
>> GossipStage                       0         0         363003         0
>> 0
>> CacheCleanupExecutor              0         0              0         0
>> 0
>> AntiEntropyStage                  0         0              0         0
>> 0
>> MigrationStage                    0         0              2         0
>> 0
>> Sampler                           0         0              0         0
>> 0
>> ValidationExecutor                0         0              0         0
>> 0
>> CommitLogArchiver                 0         0              0         0
>> 0
>> MiscStage                         0         0              0         0
>> 0
>> MemtableFlushWriter               0         0          32644         0
>> 0
>> MemtableReclaimMemory             0         0          32644         0
>> 0
>> PendingRangeCalculator            0         0            527         0
>> 0
>> MemtablePostFlush                 0         0          36565         0
>> 0
>> CompactionExecutor                2        70         108621         0
>> 0
>> InternalResponseStage             0         0              0         0
>> 0
>> HintedHandoff                     0         0             10         0
>> 0
>> Native-Transport-Requests         6         0      188996929         0
>> 79122
>>
>> Message type           Dropped
>> RANGE_SLICE                  0
>> READ_REPAIR                  0
>> PAGED_RANGE                  0
>> BINARY                       0
>> READ                         0
>> MUTATION                     0
>> _TRACE                       0
>> REQUEST_RESPONSE             0
>> COUNTER_MUTATION             0
>>
>> - If I do a nodetool drain on such nodes, and then wait for a while, the
>> number of pending compactions stays high even if there are no compactions
>> being executed anymore and the node is completely idle:
>>
>> $ nodetool compactionstats
>> pending tasks: 128
>>
>> - It's also interesting to notice how the compaction in the previous
>> example is trying to compact ~37 GB, which is essentially the whole size of
>> the column family message_data1 as reported by cfstats:
>>
>> $ nodetool cfstats -H draios.message_data1
>> Keyspace: draios
>> Read Count: 208168
>> Read Latency: 2.4791508685292647 ms.
>> Write Count: 502529
>> Write Latency: 0.20701542000561163 ms.
>> Pending Flushes: 0
>> Table: message_data1
>> SSTable count: 261
>> SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
>> Space used (live): 36.98 GB
>> Space used (total): 36.98 GB
>> Space used by snapshots (total): 0 bytes
>> Off heap memory used (total): 36.21 MB
>> SSTable Compression Ratio: 0.15461126176169512
>> Number of keys (estimate): 101025
>> Memtable cell count: 229344
>> Memtable data size: 82.4 MB
>> Memtable off heap memory used: 0 bytes
>> Memtable switch count: 83
>> Local read count: 208225
>> Local read latency: 2.479 ms
>> Local write count: 502581
>> Local write latency: 0.208 ms
>> Pending flushes: 0
>> Bloom filter false positives: 11497
>> Bloom filter false ratio: 0.04307
>> Bloom filter space used: 94.97 KB
>> Bloom filter off heap memory used: 92.93 KB
>> Index summary off heap memory used: 57.88 KB
>> Compression metadata off heap memory used: 36.06 MB
>> Compacted partition minimum bytes: 447 bytes
>> Compacted partition maximum bytes: 34.48 MB
>> Compacted partition mean bytes: 1.51 MB
>> Average live cells per slice (last five minutes): 26.269698643294515
>> Maximum live cells per slice (last five minutes): 100.0
>> Average tombstones per slice (last five minutes): 0.0
>> Maximum tombstones per slice (last five minutes): 0.0
>>
>> - There are no warnings or errors in the log, even after a clean restart
>>
>> - Restarting the node doesn't seem to have any effect on the number of
>> pending compactions
>>
>> Any help would be very appreciated.
>>
>> Thank you for reading
>>
>



-- 
Close the World, Open the Net
http://www.linux-wizard.net

Re: Pending compactions not going down on some nodes of the cluster

Posted by Gianluca Borello <gi...@sysdig.com>.
Thank you for your reply, definitely appreciate the tip on the compressed
size.

I understand your point, in fact whenever we bootstrap a new node we see a
huge number of pending compactions (in the order of thousands), and they
usually decrease steadily until they reach 0 in just a few hours. With this
node, however, we are way beyond that point, it has been 3 days since the
number of pending compaction started fluctuating around ~150 without any
sign of going down (I can see from Opscenter it's almost a straight line
starting a few hours after the bootstrap). In particular, to reply to your
point:

- The number of sstables for this CF on this node is around 250, which is
in the same range of all the other nodes in the cluster (I counted the
number on each one of them, and every node is in the 200-400 range)

- This theory doesn't seem to explain why, when doing "nodetool drain", the
compactions completely stop after a few minutes and I get something such as:

$ nodetool compactionstats
pending tasks: 128

So no compactions being executed (since there is no more write activity),
but the pending number is still high.

Thanks again


On Mon, Mar 21, 2016 at 3:19 PM, Jeff Jirsa <je...@crowdstrike.com>
wrote:

> > We added a bunch of new nodes to a cluster (2.1.13) and everything went
> fine, except for the number of pending compactions that is staying quite
> high on a subset of the new nodes. Over the past 3 days, the pending
> compactions have never been less than ~130 on such nodes, with peaks of
> ~200.
>
> When you bootstrap with Vnodes, you end up with thousands (or tens of
> thousands) of sstables – with 256 Vnodes (default) * 20 sstables per node,
> your resulting node will have 5k sstables. It takes quite a while for
> compaction to chew through that. If you added a bunch of nodes in sequence,
> you’d have 5k on the first node, then potentailly 10k on the next, and
> could potentially keep increasing as you start streaming from nodes that
> have way too many sstables.  This is one of the reasons that many people
> who have to grow their clusters frequently try not to use vnodes.
>
> From your other email:
>
> > Also related to this point, now I'm seeing something even more odd:
> some compactions are way bigger than the size of the column family itself,
> such as:
>
> The size reported by compactionstats is the uncompressed size – if you’re
> using compression, it’s perfectly reasonable for 30G of data to show up as
> 118G of data during compaction.
>
> - Jeff
>
> From: Gianluca Borello
> Reply-To: "user@cassandra.apache.org"
> Date: Monday, March 21, 2016 at 12:50 PM
> To: "user@cassandra.apache.org"
> Subject: Pending compactions not going down on some nodes of the cluster
>
> Hi,
>
> We added a bunch of new nodes to a cluster (2.1.13) and everything went
> fine, except for the number of pending compactions that is staying quite
> high on a subset of the new nodes. Over the past 3 days, the pending
> compactions have never been less than ~130 on such nodes, with peaks of
> ~200. On the other nodes, they correctly fluctuate between 0 and ~20, which
> has been our norm for a long time.
>
> We are quite paranoid about pending compactions because in the past such
> high number caused a lot of data being brought in memory during some reads
> and that triggered a chain reaction of full GCs that brought down our
> cluster, so we try to monitor them closely.
>
> Some data points that should let the situation speak for itself:
>
> - We use LCS for all our column families
>
> - The cluster is operating absolutely fine and seems healthy, and every
> node is handling pretty much the same load in terms of reads and writes.
> Also, these nodes with higher pending compactions don't seem in any way
> performing worse than the others
>
> - The pending compactions don't go down even when setting the compaction
> throughput to unlimited for a very long time
>
> - This is the typical output of compactionstats and tpstats:
>
> $ nodetool compactionstats
> pending tasks: 137
>    compaction type   keyspace            table     completed         total
>    unit   progress
>         Compaction     draios   message_data60    6111208394    6939536890
>   bytes     88.06%
>         Compaction     draios    message_data1   26473390790   37243294809
>   bytes     71.08%
> Active compaction remaining time :        n/a
>
> $ nodetool tpstats
> Pool Name                    Active   Pending      Completed   Blocked
>  All time blocked
> CounterMutationStage              0         0              0         0
>             0
> ReadStage                         1         0      111766844         0
>             0
> RequestResponseStage              0         0      244259493         0
>             0
> MutationStage                     0         0      163268653         0
>             0
> ReadRepairStage                   0         0        8933323         0
>             0
> GossipStage                       0         0         363003         0
>             0
> CacheCleanupExecutor              0         0              0         0
>             0
> AntiEntropyStage                  0         0              0         0
>             0
> MigrationStage                    0         0              2         0
>             0
> Sampler                           0         0              0         0
>             0
> ValidationExecutor                0         0              0         0
>             0
> CommitLogArchiver                 0         0              0         0
>             0
> MiscStage                         0         0              0         0
>             0
> MemtableFlushWriter               0         0          32644         0
>             0
> MemtableReclaimMemory             0         0          32644         0
>             0
> PendingRangeCalculator            0         0            527         0
>             0
> MemtablePostFlush                 0         0          36565         0
>             0
> CompactionExecutor                2        70         108621         0
>             0
> InternalResponseStage             0         0              0         0
>             0
> HintedHandoff                     0         0             10         0
>             0
> Native-Transport-Requests         6         0      188996929         0
>         79122
>
> Message type           Dropped
> RANGE_SLICE                  0
> READ_REPAIR                  0
> PAGED_RANGE                  0
> BINARY                       0
> READ                         0
> MUTATION                     0
> _TRACE                       0
> REQUEST_RESPONSE             0
> COUNTER_MUTATION             0
>
> - If I do a nodetool drain on such nodes, and then wait for a while, the
> number of pending compactions stays high even if there are no compactions
> being executed anymore and the node is completely idle:
>
> $ nodetool compactionstats
> pending tasks: 128
>
> - It's also interesting to notice how the compaction in the previous
> example is trying to compact ~37 GB, which is essentially the whole size of
> the column family message_data1 as reported by cfstats:
>
> $ nodetool cfstats -H draios.message_data1
> Keyspace: draios
> Read Count: 208168
> Read Latency: 2.4791508685292647 ms.
> Write Count: 502529
> Write Latency: 0.20701542000561163 ms.
> Pending Flushes: 0
> Table: message_data1
> SSTable count: 261
> SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
> Space used (live): 36.98 GB
> Space used (total): 36.98 GB
> Space used by snapshots (total): 0 bytes
> Off heap memory used (total): 36.21 MB
> SSTable Compression Ratio: 0.15461126176169512
> Number of keys (estimate): 101025
> Memtable cell count: 229344
> Memtable data size: 82.4 MB
> Memtable off heap memory used: 0 bytes
> Memtable switch count: 83
> Local read count: 208225
> Local read latency: 2.479 ms
> Local write count: 502581
> Local write latency: 0.208 ms
> Pending flushes: 0
> Bloom filter false positives: 11497
> Bloom filter false ratio: 0.04307
> Bloom filter space used: 94.97 KB
> Bloom filter off heap memory used: 92.93 KB
> Index summary off heap memory used: 57.88 KB
> Compression metadata off heap memory used: 36.06 MB
> Compacted partition minimum bytes: 447 bytes
> Compacted partition maximum bytes: 34.48 MB
> Compacted partition mean bytes: 1.51 MB
> Average live cells per slice (last five minutes): 26.269698643294515
> Maximum live cells per slice (last five minutes): 100.0
> Average tombstones per slice (last five minutes): 0.0
> Maximum tombstones per slice (last five minutes): 0.0
>
> - There are no warnings or errors in the log, even after a clean restart
>
> - Restarting the node doesn't seem to have any effect on the number of
> pending compactions
>
> Any help would be very appreciated.
>
> Thank you for reading
>
>

Re: Pending compactions not going down on some nodes of the cluster

Posted by Jeff Jirsa <je...@crowdstrike.com>.
> We added a bunch of new nodes to a cluster (2.1.13) and everything went fine, except for the number of pending compactions that is staying quite high on a subset of the new nodes. Over the past 3 days, the pending compactions have never been less than ~130 on such nodes, with peaks of ~200.

When you bootstrap with Vnodes, you end up with thousands (or tens of thousands) of sstables – with 256 Vnodes (default) * 20 sstables per node, your resulting node will have 5k sstables. It takes quite a while for compaction to chew through that. If you added a bunch of nodes in sequence, you’d have 5k on the first node, then potentailly 10k on the next, and could potentially keep increasing as you start streaming from nodes that have way too many sstables.  This is one of the reasons that many people who have to grow their clusters frequently try not to use vnodes.

>From your other email:

> Also related to this point, now I'm seeing something even more odd: some compactions are way bigger than the size of the column family itself, such as:

The size reported by compactionstats is the uncompressed size – if you’re using compression, it’s perfectly reasonable for 30G of data to show up as 118G of data during compaction.

- Jeff

From:  Gianluca Borello
Reply-To:  "user@cassandra.apache.org"
Date:  Monday, March 21, 2016 at 12:50 PM
To:  "user@cassandra.apache.org"
Subject:  Pending compactions not going down on some nodes of the cluster

Hi,

We added a bunch of new nodes to a cluster (2.1.13) and everything went fine, except for the number of pending compactions that is staying quite high on a subset of the new nodes. Over the past 3 days, the pending compactions have never been less than ~130 on such nodes, with peaks of ~200. On the other nodes, they correctly fluctuate between 0 and ~20, which has been our norm for a long time.

We are quite paranoid about pending compactions because in the past such high number caused a lot of data being brought in memory during some reads and that triggered a chain reaction of full GCs that brought down our cluster, so we try to monitor them closely.

Some data points that should let the situation speak for itself:

- We use LCS for all our column families

- The cluster is operating absolutely fine and seems healthy, and every node is handling pretty much the same load in terms of reads and writes. Also, these nodes with higher pending compactions don't seem in any way performing worse than the others

- The pending compactions don't go down even when setting the compaction throughput to unlimited for a very long time

- This is the typical output of compactionstats and tpstats:

$ nodetool compactionstats
pending tasks: 137
   compaction type   keyspace            table     completed         total    unit   progress
        Compaction     draios   message_data60    6111208394    6939536890   bytes     88.06%
        Compaction     draios    message_data1   26473390790   37243294809   bytes     71.08%
Active compaction remaining time :        n/a

$ nodetool tpstats
Pool Name                    Active   Pending      Completed   Blocked  All time blocked
CounterMutationStage              0         0              0         0                 0
ReadStage                         1         0      111766844         0                 0
RequestResponseStage              0         0      244259493         0                 0
MutationStage                     0         0      163268653         0                 0
ReadRepairStage                   0         0        8933323         0                 0
GossipStage                       0         0         363003         0                 0
CacheCleanupExecutor              0         0              0         0                 0
AntiEntropyStage                  0         0              0         0                 0
MigrationStage                    0         0              2         0                 0
Sampler                           0         0              0         0                 0
ValidationExecutor                0         0              0         0                 0
CommitLogArchiver                 0         0              0         0                 0
MiscStage                         0         0              0         0                 0
MemtableFlushWriter               0         0          32644         0                 0
MemtableReclaimMemory             0         0          32644         0                 0
PendingRangeCalculator            0         0            527         0                 0
MemtablePostFlush                 0         0          36565         0                 0
CompactionExecutor                2        70         108621         0                 0
InternalResponseStage             0         0              0         0                 0
HintedHandoff                     0         0             10         0                 0
Native-Transport-Requests         6         0      188996929         0             79122

Message type           Dropped
RANGE_SLICE                  0
READ_REPAIR                  0
PAGED_RANGE                  0
BINARY                       0
READ                         0
MUTATION                     0
_TRACE                       0
REQUEST_RESPONSE             0
COUNTER_MUTATION             0

- If I do a nodetool drain on such nodes, and then wait for a while, the number of pending compactions stays high even if there are no compactions being executed anymore and the node is completely idle: 

$ nodetool compactionstats
pending tasks: 128

- It's also interesting to notice how the compaction in the previous example is trying to compact ~37 GB, which is essentially the whole size of the column family message_data1 as reported by cfstats:

$ nodetool cfstats -H draios.message_data1
Keyspace: draios
Read Count: 208168
Read Latency: 2.4791508685292647 ms.
Write Count: 502529
Write Latency: 0.20701542000561163 ms.
Pending Flushes: 0
Table: message_data1
SSTable count: 261
SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
Space used (live): 36.98 GB
Space used (total): 36.98 GB
Space used by snapshots (total): 0 bytes
Off heap memory used (total): 36.21 MB
SSTable Compression Ratio: 0.15461126176169512
Number of keys (estimate): 101025
Memtable cell count: 229344
Memtable data size: 82.4 MB
Memtable off heap memory used: 0 bytes
Memtable switch count: 83
Local read count: 208225
Local read latency: 2.479 ms
Local write count: 502581
Local write latency: 0.208 ms
Pending flushes: 0
Bloom filter false positives: 11497
Bloom filter false ratio: 0.04307
Bloom filter space used: 94.97 KB
Bloom filter off heap memory used: 92.93 KB
Index summary off heap memory used: 57.88 KB
Compression metadata off heap memory used: 36.06 MB
Compacted partition minimum bytes: 447 bytes
Compacted partition maximum bytes: 34.48 MB
Compacted partition mean bytes: 1.51 MB
Average live cells per slice (last five minutes): 26.269698643294515
Maximum live cells per slice (last five minutes): 100.0
Average tombstones per slice (last five minutes): 0.0
Maximum tombstones per slice (last five minutes): 0.0

- There are no warnings or errors in the log, even after a clean restart

- Restarting the node doesn't seem to have any effect on the number of pending compactions

Any help would be very appreciated.

Thank you for reading