On 19 Nov 2014, at 19:53, Robert Coli <[email protected]> wrote:
>
> My hunch is that you originally triggered this by picking up some obsolete
> SSTables during the 1.2 era. Probably if you clean up the existing zombies
> you will not encounter them again, unless you encounter another "obsolete
> sstables marked live" bug. I agree that your compaction exceptions in the 1.2
> era are likely implicated.
I’m still in the “1.2 era”, I upgraded from 1.2.16 to 1.2.19. You mentioned
that obsolete sstables may have been picked up, and each node had 3-9
exceptions like this when they were brought down prior to being updated:
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)
Can it be that they were all in the middle of a compaction (Leveled compaction)
and the new sstables were written but the old ones were not deleted? Will
Cassandra blindly pick up old and new sstables when it restarts?
If so, I have a few questions:
1- What is the correct sequence of commands to bring down a node safely? I know
that “drain" was used here, because it is in the log. I’ve read somewhere that
drain should not be used and “disablethrift”, “disablegossip”, “flush” and then
waiting a while is the correct way.
2- Why won’t repair propagate this column value to the other nodes? Repairs
have run everyday and the value is still missing on the other nodes.
3- If I have these rogue sstables loaded this seems like a time bomb. Down the
line I will again delete columns that will reappear after some time. Is there a
way I can find these sstables that should not be there? I thought the timestamp
of the file would help but this zombie column is present on one of the latest
sstables.
Thanks,
André