cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alain RODRIGUEZ (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-9862) Missing sstable files seen some time after the same file is logged as being part of a compaction
Date Fri, 28 Aug 2015 13:20:46 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-9862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14718547#comment-14718547
] 

Alain RODRIGUEZ commented on CASSANDRA-9862:
--------------------------------------------

After a rolling restart I sometime face this.

My guess so far is that this file is taken into consideration for a compaction, then something
weird happen, the file vanishes.
Then compactions always try to use this file to compact and stay stuck for a given column
family...

About the "outside world" we are on AWS, and using the image provided by datastax https://github.com/riptano/ComboAMI
to build our nodes (+ some more configuration).

[~yukim] you might want to see:
Error : https://gist.github.com/arodrime/e6a1bfd8e6eb3be30d0f
Compactions in progress: https://gist.github.com/arodrime/eef771f9a0d62709be95

Those are consequences of the bug. Not sure about what causes it nor about how to restore
the node (I use to truncate the compactions_in_progress table and restart the node when I
face this). I am keeping this node "ill" for 2 days for Datastax support but I guess this
 node will explode very soon due to a "too many file exception" sadly. If you are quick answering
this I might perform action you'll tell me.

I hope this will help.

Also, using 2.0.16.

I'll try to set log4j.logger.org.apache.cassandra.db.compaction=DEBUG on every node and restart
all the nodes. I just have to make sure our production won't suffer more from all this...

I'll let you know.

> 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)

Mime
View raw message