Arya Goudarzi created CASSANDRA-5266:
----------------------------------------
Summary: FileNotFoundException during AntiEntropy Stream Session
after compaction
Key: CASSANDRA-5266
URL: https://issues.apache.org/jira/browse/CASSANDRA-5266
Project: Cassandra
Issue Type: Bug
Components: Core
Affects Versions: 1.1.6
Environment: Ubuntu 10.04
Cassandra 1.1.6
Jave 6 (64 Bit Sun JVM)
Reporter: Arya Goudarzi
It seems that there is a race condition in the compaction process vs
AntiEntropy Streams. We often have repairs tossing lots of exceptions in the
logs as follows:
java.lang.RuntimeException: java.io.FileNotFoundException:
/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20056-Data.db
(No such file or directory)
at
org.apache.cassandra.utils.FBUtilities.unchecked(FBUtilities.java:628)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)
Caused by: java.io.FileNotFoundException:
/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20056-Data.db
(No such file or directory)
at java.io.RandomAccessFile.open(Native Method)
at java.io.RandomAccessFile.<init>(RandomAccessFile.java:216)
at
org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:67)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.<init>(CompressedRandomAccessReader.java:64)
at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:46)
at
org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
at
org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:94)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
... 3 more
Now about 1-2 minutes before that, I see Secondary Index buildup and compaction
which does compact the same SSTable to a new one.
INFO [Thread-90838] 2013-02-07 13:51:58,362 SecondaryIndexManager.java (line
126) Submitting index build of publisher_id, for data in
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20054-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20055-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20056-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20057-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20058-Data.db')
INFO [CompactionExecutor:436] 2013-02-07 13:51:58,363 CompactionTask.java
(line 109) Compacting
[SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20055-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20054-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20058-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20057-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/keyspace_production/User/keyspace_production-User-hf-20056-Data.db')]
Now, does this make the repair sessions to fail? It seems the node was trying
to stream a file to a node being repaired but apparently it did not.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira