Hi Anthony,
  We have seen similar issues that we filed under CASSANDRA-452.  I've
filed a ticket CASSANDRA-453 which won't resolve the issue, but can
help figure out which sstables are corrupt.

In order to run apply the patch to trunk compile  and run with the following:
bin/sstabletool validate /home/sammy/corrupt/Digg/*-Data.db

Please note if you copied onto a different machine, you will need to
have the sstable in a parent directory with the keyspace name.

Cheers,
Sammy


On Wed, Sep 30, 2009 at 11:24 AM, Anthony Molinaro
<antho...@alumni.caltech.edu> wrote:
> Hi,
>
> I'm not getting any responses on IRC, so figured I'd put this out on
> the mailing list.
>
> I had a 3 node cassandra cluster, replication factor 3 on
> 3 ec2 m1.large instances behind an haproxy.  I restarted one
> of the node to test out some modified sysctl's (tcp stack tuning).
> As soon as I restarted it the other 2 nodes started spiking memory
> use and the first node seemed to have corrupted data.  The corruption
> is an exception when I read some and only some keys.
>
> The exception is
>
> ERROR [pool-1-thread-1] 2009-09-30 17:50:30,037 Cassandra.java (line 679) 
> Internal error processing get_slice
> java.lang.RuntimeException: java.io.EOFException
>        at 
> org.apache.cassandra.service.CassandraServer.readColumnFamily(CassandraServer.java:104)
>        at 
> org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
>        at 
> org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
>        at 
> org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
>        at 
> org.apache.cassandra.service.Cassandra$Processor$get_slice.process(Cassandra.java:671)
>        at 
> org.apache.cassandra.service.Cassandra$Processor.process(Cassandra.java:627)
>        at 
> org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:253)
>        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.io.EOFException
>        at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725)
>        at 
> org.apache.cassandra.io.IndexHelper.deserializeIndex(IndexHelper.java:95)
>        at 
> org.apache.cassandra.db.filter.SSTableSliceIterator$ColumnGroupReader.<init>(SSTableSliceIterator.java:118)
>        at 
> org.apache.cassandra.db.filter.SSTableSliceIterator.<init>(SSTableSliceIterator.java:56)
>        at 
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:64)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1390)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1325)
>        at org.apache.cassandra.db.Table.getRow(Table.java:590)
>        at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:59)
>        at 
> org.apache.cassandra.service.StorageProxy.weakReadLocal(StorageProxy.java:471)
>        at 
> org.apache.cassandra.service.StorageProxy.readProtocol(StorageProxy.java:309)
>        at 
> org.apache.cassandra.service.CassandraServer.readColumnFamily(CassandraServer.java:100)
>        ... 9 more
>
>
> I ended up having to fire up some new instances, and reload the data
> (luckily this is my small instance which I can reload quickly, I've got a
> large cassandra cluster currently loading which I will not be
> able to do this with, so I'm a little scared about that cluster).
>
> Anyway, any ideas?  I've left the broken cluster so I can 
> investigate/patch/etc.
>
> -Anthony
>
> --
> ------------------------------------------------------------------------
> Anthony Molinaro                           <antho...@alumni.caltech.edu>
>

Attachment: 0001-CASSANDRA-453-Added-initial-version-of-the-sstable-t.patch
Description: Binary data

Reply via email to