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

Gary Dusbabek commented on CASSANDRA-1477:
------------------------------------------

The exception happens because the compaction marker exists but the deletion 
executor hasn't done its thing.  It's easy to visualize:
* A keyspace is created along with a column family.  sstables are written.
* That keyspace is dropped. compacted markers are written.
* That keyspace and column family are recreated.  Meanwhile, no files have been 
deleted.
* As part of re-Table.open()ing it, CFS.scrubDataDirectories() is called but 
doesn't scrub the compacted markers because of those are now excluded by 
files() * (I'm pretty sure this wasn't working, or wasn't working like we 
expect, in the first place, since B.Todd was getting the error before applying 
the patch).
* Newly created sstables match the names of the compaction markers and will be 
possibly deleted if timing is bad.

Two fixes come to mind:
1. keep the first patch that has already been committed and make AddKeyspace 
ensure that data directories, if they exist, are empty.
2. revert the last patch and make AddColumnFamily ensure that its data 
directory, if it exists, is empty.

I favor option 2.

> drop/recreate column family race condition
> ------------------------------------------
>
>                 Key: CASSANDRA-1477
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1477
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.7 beta 2
>         Environment: 1 Node cluster, latest code from 0.7 trunk
>            Reporter: B. Todd Burruss
>            Assignee: Gary Dusbabek
>            Priority: Blocker
>             Fix For: 0.7 beta 2
>
>         Attachments: RaceConditionTest.java, 
> v1-0001-ensure-that-compacted-sstables-are-excluded-from-newly.txt
>
>
> using 0.7 latest from trunk as of few minutes ago.  1 client, 1 node
> i have the scenario where i want to drop a column family and recreate it 
> - unit testing for instance, is a good reason you may want to do this 
> (always start fresh).
> the problem i observe is that if i do the following:
> 1 - drop the column family
> 2 - recreate it
> 3 - read data from a key that existed before dropping, but doesn't exist now
> if those steps happen fast enough, i will get the old row - definitely 
> no good.
> if they happen slow enough, get_slice throws:
> "org.apache.thrift.TApplicationException: Internal error processing 
> get_slice"
> .. and on the server i see:
> 2010-09-07 13:53:48,086 ERROR 
> [org.apache.cassandra.thrift.Cassandra$Processor] (pool-1-thread-4:) - 
> Internal error processing get_slice
> java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
> java.io.IOError: java.io.FileNotFoundException: 
> cassandra-data/data/Queues/test_1283892789285_Waiting-e-1-Data.db (No 
> such file or directory)
>      at 
> org.apache.cassandra.service.StorageProxy.weakRead(StorageProxy.java:275)
>      at 
> org.apache.cassandra.service.StorageProxy.readProtocol(StorageProxy.java:218)
>      at 
> org.apache.cassandra.thrift.CassandraServer.readColumnFamily(CassandraServer.java:114)
>      at 
> org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServer.java:220)
>      at 
> org.apache.cassandra.thrift.CassandraServer.multigetSliceInternal(CassandraServer.java:299)
>      at 
> org.apache.cassandra.thrift.CassandraServer.get_slice(CassandraServer.java:260)
>      at 
> org.apache.cassandra.thrift.Cassandra$Processor$get_slice.process(Cassandra.java:2795)
>      at 
> org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2651)
>      at 
> org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:167)
>      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:619)
> Caused by: java.util.concurrent.ExecutionException: java.io.IOError: 
> java.io.FileNotFoundException: 
> cassandra-data/data/Queues/test_1283892789285_Waiting-e-1-Data.db (No 
> such file or directory)
>      at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
>      at java.util.concurrent.FutureTask.get(FutureTask.java:83)
>      at 
> org.apache.cassandra.service.StorageProxy.weakRead(StorageProxy.java:271)
>      ... 11 more
> Caused by: java.io.IOError: java.io.FileNotFoundException: 
> cassandra-data/data/Queues/test_1283892789285_Waiting-e-1-Data.db (No 
> such file or directory)
>      at 
> org.apache.cassandra.io.util.BufferedSegmentedFile.getSegment(BufferedSegmentedFile.java:68)
>      at 
> org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:509)
>      at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:49)
>      at 
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:65)
>      at 
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:76)
>      at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:961)
>      at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:856)
>      at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:826)
>      at org.apache.cassandra.db.Table.getRow(Table.java:321)
>      at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63)
>      at 
> org.apache.cassandra.service.StorageProxy$weakReadLocalCallable.call(StorageProxy.java:737)
>      at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>      at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>      ... 3 more
> Caused by: java.io.FileNotFoundException: 
> cassandra-data/data/Queues/test_1283892789285_Waiting-e-1-Data.db (No 
> such file or directory)
>      at java.io.RandomAccessFile.open(Native Method)
>      at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
>      at java.io.RandomAccessFile.<init>(RandomAccessFile.java:98)
>      at 
> org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:142)
>      at 
> org.apache.cassandra.io.util.BufferedSegmentedFile.getSegment(BufferedSegmentedFile.java:62)
>      ... 15 more

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to