You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Sotirios Delimanolis <so...@yahoo.com.INVALID> on 2017/10/24 21:18:44 UTC

Index summary redistribution seems to block all compactions

On a Cassandra 2.2.11 cluster, I noticed estimated compactions accumulating on one node. nodetool compactionstats showed the following:
                compaction type    keyspace         table   completed       total    unit   progress                     Compaction         ks1    some_table   204.68 MB   204.98 MB   bytes     99.86%   Index summary redistribution        null          null   457.72 KB      950 MB   bytes      0.05%                     Compaction         ks1    some_table   461.61 MB   461.95 MB   bytes     99.93%           Tombstone Compaction         ks1    some_table   618.34 MB   618.47 MB   bytes     99.98%                     Compaction         ks1    some_table   378.37 MB      380 MB   bytes     99.57%           Tombstone Compaction         ks1    some_table   326.51 MB   327.63 MB   bytes     99.66%           Tombstone Compaction         ks2   other_table    29.38 MB    29.38 MB   bytes    100.00%           Tombstone Compaction         ks1    some_table    503.4 MB   507.28 MB   bytes     99.24%                     Compaction         ks1    some_table   353.44 MB   353.47 MB   bytes     99.99%

They had been like this for a while (all different tables). A thread dump showed all 8 CompactionExecutor threads looking like
"CompactionExecutor:6" #84 daemon prio=1 os_prio=4 tid=0x00007f5771172000 nid=0x7646 waiting on condition [0x00007f578847b000]   java.lang.Thread.State: WAITING (parking)        at sun.misc.Unsafe.park(Native Method)        - parking to wait for  <0x00000005fe5656e8> (a com.google.common.util.concurrent.AbstractFuture$Sync)        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)        at com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:285)        at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)        at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:390)        at org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(SystemKeyspace.java:593)        at org.apache.cassandra.db.SystemKeyspace.finishCompaction(SystemKeyspace.java:368)        at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:205)        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)        at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:74)        at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:80)        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:257)        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)        at java.util.concurrent.FutureTask.run(FutureTask.java:266)        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)        at java.lang.Thread.run(Thread.java:745)  
A MemtablePostFlush thread was awaiting some flush count down latch
"MemtablePostFlush:1" #30 daemon prio=5 os_prio=0 tid=0x00007f57705dac00 nid=0x75bf waiting on condition [0x00007f578a8fb000]   java.lang.Thread.State: WAITING (parking)        at sun.misc.Unsafe.park(Native Method)        - parking to wait for  <0x0000000573da6c90> (a java.util.concurrent.CountDownLatch$Sync)        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)        at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)        at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:1073)        at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:1026)        at java.util.concurrent.FutureTask.run(FutureTask.java:266)        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)        at java.lang.Thread.run(Thread.java:745)  

The 4 MemtableFlushWriter threads were all RUNNABLE, sorting something in IntervalTree. Finally, the IndexSummaryManager thread was also RUNNABLE:
"IndexSummaryManager:1" #1463 daemon prio=1 os_prio=4 tid=0x00007f577139b000 nid=0x8100 runnable [0x00007f5726f6c000]   java.lang.Thread.State: RUNNABLE        at com.google.common.collect.ImmutableSet.construct(ImmutableSet.java:206)        at com.google.common.collect.ImmutableSet.copyOf(ImmutableSet.java:375)        at org.apache.cassandra.db.lifecycle.Helpers.replace(Helpers.java:43)        at org.apache.cassandra.db.lifecycle.View$2.apply(View.java:166)        at org.apache.cassandra.db.lifecycle.View$2.apply(View.java:161)        at org.apache.cassandra.db.lifecycle.Tracker.apply(Tracker.java:138)        at org.apache.cassandra.db.lifecycle.Tracker.apply(Tracker.java:111)        at org.apache.cassandra.db.lifecycle.Tracker.apply(Tracker.java:118)        at org.apache.cassandra.db.lifecycle.LifecycleTransaction.unmarkCompacting(LifecycleTransaction.java:445)        at org.apache.cassandra.db.lifecycle.LifecycleTransaction.cancel(LifecycleTransaction.java:400)        at org.apache.cassandra.io.sstable.IndexSummaryRedistribution.adjustSamplingLevels(IndexSummaryRedistribution.java:230)        at org.apache.cassandra.io.sstable.IndexSummaryRedistribution.redistributeSummaries(IndexSummaryRedistribution.java:126)        at org.apache.cassandra.db.compaction.CompactionManager.runIndexSummaryRedistribution(CompactionManager.java:1400)        at org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries(IndexSummaryManager.java:250)        at org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries(IndexSummaryManager.java:228)        at org.apache.cassandra.io.sstable.IndexSummaryManager$1.runMayThrow(IndexSummaryManager.java:125)        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)        at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)        at java.lang.Thread.run(Thread.java:745)  
How should I interpret these? What flushing behavior is blocking the compactions? 

Re: Index summary redistribution seems to block all compactions

Posted by Sotirios Delimanolis <so...@yahoo.com.INVALID>.
 Nothing obvious in the logs, and nothing seemingly related to the details in that ticket. We don't perform any scrubs either (haven't on this node recently at least).    On Wednesday, October 25, 2017, 1:08:38 AM PDT, Marcus Eriksson <kr...@gmail.com> wrote:  
 
 Anything in the logs? It *could* be https://issues.apache.org/jira/browse/CASSANDRA-13873
On Tue, Oct 24, 2017 at 11:18 PM, Sotirios Delimanolis <so...@yahoo.com.invalid> wrote:

On a Cassandra 2.2.11 cluster, I noticed estimated compactions accumulating on one node. nodetool compactionstats showed the following:
                compaction type    keyspace         table   completed       total    unit   progress                     Compaction         ks1    some_table   204.68 MB   204.98 MB   bytes     99.86%   Index summary redistribution        null          null   457.72 KB      950 MB   bytes      0.05%                     Compaction         ks1    some_table   461.61 MB   461.95 MB   bytes     99.93%           Tombstone Compaction         ks1    some_table   618.34 MB   618.47 MB   bytes     99.98%                     Compaction         ks1    some_table   378.37 MB      380 MB   bytes     99.57%           Tombstone Compaction         ks1    some_table   326.51 MB   327.63 MB   bytes     99.66%           Tombstone Compaction         ks2   other_table    29.38 MB    29.38 MB   bytes    100.00%           Tombstone Compaction         ks1    some_table    503.4 MB   507.28 MB   bytes     99.24%                     Compaction         ks1    some_table   353.44 MB   353.47 MB   bytes     99.99%

They had been like this for a while (all different tables). A thread dump showed all 8 CompactionExecutor threads looking like
"CompactionExecutor:6" #84 daemon prio=1 os_prio=4 tid=0x00007f5771172000 nid=0x7646 waiting on condition [0x00007f578847b000]   java.lang.Thread.State: WAITING (parking)        at sun.misc.Unsafe.park(Native Method)        - parking to wait for  <0x00000005fe5656e8> (a com.google.common.util. concurrent.AbstractFuture$ Sync)        at java.util.concurrent.locks. LockSupport.park(LockSupport. java:175)        at java.util.concurrent.locks. AbstractQueuedSynchronizer. parkAndCheckInterrupt( AbstractQueuedSynchronizer. java:836)        at java.util.concurrent.locks. AbstractQueuedSynchronizer. doAcquireSharedInterruptibly( AbstractQueuedSynchronizer. java:997)        at java.util.concurrent.locks. AbstractQueuedSynchronizer. acquireSharedInterruptibly( AbstractQueuedSynchronizer. java:1304)        at com.google.common.util. concurrent.AbstractFuture$ Sync.get(AbstractFuture.java: 285)        at com.google.common.util. concurrent.AbstractFuture.get( AbstractFuture.java:116)        at org.apache.cassandra.utils. FBUtilities.waitOnFuture( FBUtilities.java:390)        at org.apache.cassandra.db. SystemKeyspace. forceBlockingFlush( SystemKeyspace.java:593)        at org.apache.cassandra.db. SystemKeyspace. finishCompaction( SystemKeyspace.java:368)        at org.apache.cassandra.db. compaction.CompactionTask. runMayThrow(CompactionTask. java:205)        at org.apache.cassandra.utils. WrappedRunnable.run( WrappedRunnable.java:28)        at org.apache.cassandra.db. compaction.CompactionTask. executeInternal( CompactionTask.java:74)        at org.apache.cassandra.db. compaction. AbstractCompactionTask. execute( AbstractCompactionTask.java: 80)        at org.apache.cassandra.db. compaction.CompactionManager$ BackgroundCompactionCandidate. run(CompactionManager.java: 257)        at java.util.concurrent. Executors$RunnableAdapter. call(Executors.java:511)        at java.util.concurrent. FutureTask.run(FutureTask. java:266)        at java.util.concurrent. ThreadPoolExecutor.runWorker( ThreadPoolExecutor.java:1142)        at java.util.concurrent. ThreadPoolExecutor$Worker.run( ThreadPoolExecutor.java:617)        at java.lang.Thread.run(Thread. java:745)  
A MemtablePostFlush thread was awaiting some flush count down latch
"MemtablePostFlush:1" #30 daemon prio=5 os_prio=0 tid=0x00007f57705dac00 nid=0x75bf waiting on condition [0x00007f578a8fb000]   java.lang.Thread.State: WAITING (parking)        at sun.misc.Unsafe.park(Native Method)        - parking to wait for  <0x0000000573da6c90> (a java.util.concurrent. CountDownLatch$Sync)        at java.util.concurrent.locks. LockSupport.park(LockSupport. java:175)        at java.util.concurrent.locks. AbstractQueuedSynchronizer. parkAndCheckInterrupt( AbstractQueuedSynchronizer. java:836)        at java.util.concurrent.locks. AbstractQueuedSynchronizer. doAcquireSharedInterruptibly( AbstractQueuedSynchronizer. java:997)        at java.util.concurrent.locks. AbstractQueuedSynchronizer. acquireSharedInterruptibly( AbstractQueuedSynchronizer. java:1304)        at java.util.concurrent. CountDownLatch.await( CountDownLatch.java:231)        at org.apache.cassandra.db. ColumnFamilyStore$PostFlush. call(ColumnFamilyStore.java: 1073)        at org.apache.cassandra.db. ColumnFamilyStore$PostFlush. call(ColumnFamilyStore.java: 1026)        at java.util.concurrent. FutureTask.run(FutureTask. java:266)        at java.util.concurrent. ThreadPoolExecutor.runWorker( ThreadPoolExecutor.java:1142)        at java.util.concurrent. ThreadPoolExecutor$Worker.run( ThreadPoolExecutor.java:617)        at java.lang.Thread.run(Thread. java:745)  

The 4 MemtableFlushWriter threads were all RUNNABLE, sorting something in IntervalTree. Finally, the IndexSummaryManager thread was also RUNNABLE:
"IndexSummaryManager:1" #1463 daemon prio=1 os_prio=4 tid=0x00007f577139b000 nid=0x8100 runnable [0x00007f5726f6c000]   java.lang.Thread.State: RUNNABLE        at com.google.common.collect. ImmutableSet.construct( ImmutableSet.java:206)        at com.google.common.collect. ImmutableSet.copyOf( ImmutableSet.java:375)        at org.apache.cassandra.db. lifecycle.Helpers.replace( Helpers.java:43)        at org.apache.cassandra.db. lifecycle.View$2.apply(View. java:166)        at org.apache.cassandra.db. lifecycle.View$2.apply(View. java:161)        at org.apache.cassandra.db. lifecycle.Tracker.apply( Tracker.java:138)        at org.apache.cassandra.db. lifecycle.Tracker.apply( Tracker.java:111)        at org.apache.cassandra.db. lifecycle.Tracker.apply( Tracker.java:118)        at org.apache.cassandra.db. lifecycle. LifecycleTransaction. unmarkCompacting( LifecycleTransaction.java:445)        at org.apache.cassandra.db. lifecycle. LifecycleTransaction.cancel( LifecycleTransaction.java:400)        at org.apache.cassandra.io. sstable. IndexSummaryRedistribution. adjustSamplingLevels( IndexSummaryRedistribution. java:230)        at org.apache.cassandra.io. sstable. IndexSummaryRedistribution. redistributeSummaries( IndexSummaryRedistribution. java:126)        at org.apache.cassandra.db. compaction.CompactionManager. runIndexSummaryRedistribution( CompactionManager.java:1400)        at org.apache.cassandra.io. sstable.IndexSummaryManager. redistributeSummaries( IndexSummaryManager.java:250)        at org.apache.cassandra.io. sstable.IndexSummaryManager. redistributeSummaries( IndexSummaryManager.java:228)        at org.apache.cassandra.io. sstable.IndexSummaryManager$1. runMayThrow( IndexSummaryManager.java:125)        at org.apache.cassandra.utils. WrappedRunnable.run( WrappedRunnable.java:28)        at org.apache.cassandra. concurrent. DebuggableScheduledThreadPoolE xecutor$UncomplainingRunnable. run( DebuggableScheduledThreadPoolE xecutor.java:118)        at java.util.concurrent. Executors$RunnableAdapter. call(Executors.java:511)        at java.util.concurrent. FutureTask.runAndReset( FutureTask.java:308)        at java.util.concurrent. ScheduledThreadPoolExecutor$ ScheduledFutureTask.access$ 301( ScheduledThreadPoolExecutor. java:180)        at java.util.concurrent. ScheduledThreadPoolExecutor$ ScheduledFutureTask.run( ScheduledThreadPoolExecutor. java:294)        at java.util.concurrent. ThreadPoolExecutor.runWorker( ThreadPoolExecutor.java:1142)        at java.util.concurrent. ThreadPoolExecutor$Worker.run( ThreadPoolExecutor.java:617)        at java.lang.Thread.run(Thread. java:745)  
How should I interpret these? What flushing behavior is blocking the compactions? 



Re: Index summary redistribution seems to block all compactions

Posted by Marcus Eriksson <kr...@gmail.com>.
Anything in the logs? It *could* be
https://issues.apache.org/jira/browse/CASSANDRA-13873

On Tue, Oct 24, 2017 at 11:18 PM, Sotirios Delimanolis <
sotodel_89@yahoo.com.invalid> wrote:

> On a Cassandra 2.2.11 cluster, I noticed estimated compactions
> accumulating on one node. nodetool compactionstats showed the following:
>
>                 compaction type    keyspace         table   completed
>  total    unit   progress
>                      Compaction         ks1    some_table   204.68 MB
>  204.98 MB   bytes     99.86%
>    Index summary redistribution        *null*          *null*   457.72
> KB      950 MB   bytes      *0.05%*
>                      Compaction         ks1    some_table   461.61 MB
>  461.95 MB   bytes     99.93%
>            Tombstone Compaction         ks1    some_table   618.34 MB
>  618.47 MB   bytes     99.98%
>                      Compaction         ks1    some_table   378.37 MB
> 380 MB   bytes     99.57%
>            Tombstone Compaction         ks1    some_table   326.51 MB
>  327.63 MB   bytes     99.66%
>            Tombstone Compaction         ks2   other_table    29.38 MB
> 29.38 MB   bytes    100.00%
>            Tombstone Compaction         ks1    some_table    503.4 MB
>  507.28 MB   bytes     99.24%
>                      Compaction         ks1    some_table   353.44 MB
>  353.47 MB   bytes     99.99%
>
>
> They had been like this for a while (all different tables). A thread dump
> showed all 8 CompactionExecutor threads looking like
>
> "CompactionExecutor:6" #84 daemon prio=1 os_prio=4 tid=0x00007f5771172000
> nid=0x7646 waiting on condition [0x00007f578847b000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000005fe5656e8> (a
> com.google.common.util.concurrent.AbstractFuture$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.
> java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.
> parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.
> doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.
> acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at com.google.common.util.concurrent.AbstractFuture$
> Sync.get(AbstractFuture.java:285)
>         at com.google.common.util.concurrent.AbstractFuture.get(
> AbstractFuture.java:116)
>         at org.apache.cassandra.utils.FBUtilities.waitOnFuture(
> FBUtilities.java:390)
>         at org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(
> SystemKeyspace.java:593)
>         at org.apache.cassandra.db.SystemKeyspace.finishCompaction(
> SystemKeyspace.java:368)
>         at org.apache.cassandra.db.compaction.CompactionTask.
> runMayThrow(CompactionTask.java:205)
>         at org.apache.cassandra.utils.WrappedRunnable.run(
> WrappedRunnable.java:28)
>         at org.apache.cassandra.db.compaction.CompactionTask.
> executeInternal(CompactionTask.java:74)
>         at org.apache.cassandra.db.compaction.AbstractCompactionTask.
> execute(AbstractCompactionTask.java:80)
>         at org.apache.cassandra.db.compaction.CompactionManager$
> BackgroundCompactionCandidate.run(CompactionManager.java:257)
>         at java.util.concurrent.Executors$RunnableAdapter.
> call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
>
> A MemtablePostFlush thread was awaiting some flush count down latch
>
> "MemtablePostFlush:1" #30 daemon prio=5 os_prio=0 tid=0x00007f57705dac00
> nid=0x75bf waiting on condition [0x00007f578a8fb000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x0000000573da6c90> (a
> java.util.concurrent.CountDownLatch$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.
> java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.
> parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.
> doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.
> acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.CountDownLatch.await(
> CountDownLatch.java:231)
>         at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.
> call(ColumnFamilyStore.java:1073)
>         at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.
> call(ColumnFamilyStore.java:1026)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
>
>
> The 4 MemtableFlushWriter threads were all RUNNABLE, sorting something in
> IntervalTree. Finally, the IndexSummaryManager thread was also RUNNABLE:
>
> "IndexSummaryManager:1" #1463 daemon prio=1 os_prio=4
> tid=0x00007f577139b000 nid=0x8100 runnable [0x00007f5726f6c000]
>    java.lang.Thread.State: RUNNABLE
>         at com.google.common.collect.ImmutableSet.construct(
> ImmutableSet.java:206)
>         at com.google.common.collect.ImmutableSet.copyOf(
> ImmutableSet.java:375)
>         at org.apache.cassandra.db.lifecycle.Helpers.replace(
> Helpers.java:43)
>         at org.apache.cassandra.db.lifecycle.View$2.apply(View.java:166)
>         at org.apache.cassandra.db.lifecycle.View$2.apply(View.java:161)
>         at org.apache.cassandra.db.lifecycle.Tracker.apply(
> Tracker.java:138)
>         at org.apache.cassandra.db.lifecycle.Tracker.apply(
> Tracker.java:111)
>         at org.apache.cassandra.db.lifecycle.Tracker.apply(
> Tracker.java:118)
>         at org.apache.cassandra.db.lifecycle.LifecycleTransaction.
> unmarkCompacting(LifecycleTransaction.java:445)
>         at org.apache.cassandra.db.lifecycle.LifecycleTransaction.cancel(
> LifecycleTransaction.java:400)
>         at org.apache.cassandra.io.sstable.IndexSummaryRedistribution.
> adjustSamplingLevels(IndexSummaryRedistribution.java:230)
>         at org.apache.cassandra.io.sstable.IndexSummaryRedistribution.
> redistributeSummaries(IndexSummaryRedistribution.java:126)
>         at org.apache.cassandra.db.compaction.CompactionManager.
> runIndexSummaryRedistribution(CompactionManager.java:1400)
>         at org.apache.cassandra.io.sstable.IndexSummaryManager.
> redistributeSummaries(IndexSummaryManager.java:250)
>         at org.apache.cassandra.io.sstable.IndexSummaryManager.
> redistributeSummaries(IndexSummaryManager.java:228)
>         at org.apache.cassandra.io.sstable.IndexSummaryManager$1.
> runMayThrow(IndexSummaryManager.java:125)
>         at org.apache.cassandra.utils.WrappedRunnable.run(
> WrappedRunnable.java:28)
>         at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolE
> xecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolE
> xecutor.java:118)
>         at java.util.concurrent.Executors$RunnableAdapter.
> call(Executors.java:511)
>         at java.util.concurrent.FutureTask.runAndReset(
> FutureTask.java:308)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
>
> How should I interpret these? What flushing behavior is blocking the
> compactions?
>
>