cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From André Cruz <andre.c...@co.sapo.pt>
Subject Re: Repair completes successfully but data is still inconsistent
Date Wed, 19 Nov 2014 13:18:25 GMT
On 19 Nov 2014, at 11:37, André Cruz <andre.cruz@co.sapo.pt> wrote:
> 
> All the nodes were restarted on 21-23 October, for the upgrade (1.2.16 -> 1.2.19)
I mentioned. The delete happened after. I should also point out that we were experiencing
problems related to CASSANDRA-4206 and CASSANDRA-7808.

Another possible cause are these exceptions I found in the log as the nodes were shutdown
and brought up with the new version:

INFO [RMI TCP Connection(270364)-10.134.101.18] 2014-10-21 15:04:00,867 StorageService.java
(line 939) DRAINED
ERROR [CompactionExecutor:15173] 2014-10-21 15:04:01,923 CassandraDaemon.java (line 191) Exception
in thread Thread[CompactionExecutor:15173,1,main]
java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@1dad30c0
rejected from org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor@555b9c78[Terminated,
pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 14052]
        at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor.reject(Unknown Source)
        at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(Unknown Source)
        at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(Unknown Source)
        at java.util.concurrent.ScheduledThreadPoolExecutor.submit(Unknown Source)
        at org.apache.cassandra.io.sstable.SSTableDeletingTask.schedule(SSTableDeletingTask.java:65)
        at org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:976)
        at org.apache.cassandra.db.DataTracker.removeOldSSTablesSize(DataTracker.java:370)
        at org.apache.cassandra.db.DataTracker.postReplace(DataTracker.java:335)
        at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:329)
        at org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:232)
        at org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:995)
        at org.apache.cassandra.db.compaction.CompactionTask.replaceCompactedSSTables(CompactionTask.java:270)
        at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:230)
        at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:58)
        at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:208)
        at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
        at java.util.concurrent.FutureTask.run(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
        at java.lang.Thread.run(Unknown Source)


Each node has 4-9 of these exceptions as it is going down after being drained. It seems Cassandra
was trying to delete an sstable. Can this be related?

Best regards,
André Cruz
Mime
View raw message