[ https://issues.apache.org/jira/browse/CASSANDRA-9862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14651854#comment-14651854 ]
Mark Curtis commented on CASSANDRA-9862: ---------------------------------------- So based on my thoughts above could it be that CompactionExecutor:5 failed to compact away the table "my_keyspace-my_table-jb-2085190-Data.db" due to the FNFE on 2015-06-03 12:25:38,717 or at least complete the action and therefore it "thinks" the file is still there but when it tried to access it later on it throws a FNFE for it because it actually did compact it but didn't? This is just me theorizing but this it what drove me to open this jira in the first place. The very fact we see the same file appear in two separate compaction messages seems odd to me as I would have thought you would only see any given sstable appear only once in and only once for a given compaction? > Missing sstable files seen some time after the same file is logged as being > part of a compaction > ------------------------------------------------------------------------------------------------ > > Key: CASSANDRA-9862 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9862 > Project: Cassandra > Issue Type: Bug > Reporter: Mark Curtis > Assignee: Yuki Morishita > > The system will sometimes see a file not found exception in a given node’s > logs. This is seen after the same file appears in a compaction statement > either earlier on in the logs. I found the following events of potential > interest: > I pulled out the missing files using this command: > {code} > grep "Caused by: java.io.FileNotFoundException:" * | awk '{print $4}' | sort > -u > files-to-search > {code} > Using the following file as an example: > my_keyspace-my_table-jb-2085190-Data.db > File is created from a flush > system.log.10 > {code} > 7549 INFO [FlushWriter:1] 2015-06-03 12:25:31,857 Memtable.java (line > 395) Completed flushing > /raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085190-Data.d > 7549 b (66804 bytes) for commitlog position > ReplayPosition(segmentId=1433333700832, position=2479196) > {code} > Next it appears in a compaction > system.log.10 > {code} > 8758 INFO [CompactionExecutor:5] 2015-06-03 12:25:37,209 > CompactionTask.java (line 115) Compacting > [SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2084993-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085164-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085190-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-... > ...<truncated>... > {code} > The next time we see the file its in file not found exceptions > {code} > system.log.9 > 53811 java.lang.RuntimeException: java.io.FileNotFoundException: > /raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085190-Data.db > (No such file or directory) > {code} > So going back to compaction executor 5 in system.log.10 > {code} > 9102 INFO [CompactionExecutor:5] 2015-06-03 12:25:38,684 > ColumnFamilyStore.java (line 794) Enqueuing flush of > Memtable-compactions_in_progress@1431151640(0/0 serialized/live bytes, 1 ops) > ... > 9107 INFO [CompactionExecutor:5] 2015-06-03 12:25:38,701 > CompactionTask.java (line 287) Compacted 32 sstables to > [/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-data_my_table-jb-2085196,]. > 1,970,986 bytes to 1,880,866 (~95% of original) in 1,492ms = 1.202234MB/s. > 13,576 total partitions merged to 11,243. Partition merge counts were > {1:9379, 2:1474, 3:324, 4:54 9107 , 5:11, 6:1, } > 9108 INFO [CompactionExecutor:5] 2015-06-03 12:25:38,702 > ColumnFamilyStore.java (line 794) Enqueuing flush of > Memtable-compactions_in_progress@493376470(357/3570 serialized/live bytes, 16 > ops) > ... > 9111 INFO [CompactionExecutor:5] 2015-06-03 12:25:38,717 > CompactionTask.java (line 115) Compacting > [SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085182-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085192-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085175-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085196-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085194-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085191-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085193-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085195-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2084985-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2085114-Data.db'), > > SSTableReader(path='/raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2084901-Data.db')] > 9112 ERROR [CompactionExecutor:5] 2015-06-03 12:25:38,718 > CassandraDaemon.java (line 199) Exception in thread > Thread[CompactionExecutor:5,1,main] > 9113 java.lang.RuntimeException: java.io.FileNotFoundException: > /raid0/cassandra/data/my_keyspace/my_table/my_keyspace-my_table-jb-2084901-Data.db > (No such file or directory) > {code} > So could it be that because the thread "CompactionExecutor:5" hit an > exception in line 9113 that the compaction in 8758 didn't complete? -- This message was sent by Atlassian JIRA (v6.3.4#6332)