[
https://issues.apache.org/jira/browse/CASSANDRA-452?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12758570#action_12758570
]
Jonathan Ellis commented on CASSANDRA-452:
------------------------------------------
17120 is corrupt, all right.
The local deletion time is 0, which is nonsense (it's generated from
System.CurrentTimeMillis) and the timestamp associated with that delete doesn't
look like your other timestamps. After that it's clearly reading nonsense and
eventually EOFs while trying to read 19988495 columns. Some debug output from
my local code:
DEBUG - key is 148447622005950731053602871503233733033:itemdiggs15891086
...
DEBUG - reading name of length 7
DEBUG - deserializing SC ironeus deleted @0/21025451015143424; reading 19988495
columns
DEBUG - deserializing subcolumn 0
DEBUG - reading name of length 27237
DEBUG - deserializing rryjamesstoneJ�� ...
I can't see how the compaction code could cause this kind of corruption. (Your
logs should show: is this even the product of a compaction? Or is it a direct
result of a memtable or BMt?)
If it is a product of compaction, do you have a snapshot of that sstable any
time prior to that compaction? Can you reproduce the bug compacting those
files?
I hate to blame hardware but there are a couple things that indicate this might
actually be caused by that. First, a localDeletionTime of zero is exactly 1
bit away from Integer.MIN_VALUE in 2's complement. All the other localDT
values are Integer.MIN_VALUE as would be expected if no deletes are done.
Second, no bytes are being skipped (which is often how you see some expected
small number of columns get huge) -- the row sizes are correct and all the keys
are readable where they should be.
I will attach a list of keys in this sstable so you can force read repair on
them. Tomorrow I will patch compaction to be able to recover from this error,
and if Sammy or Chris can do CASSANDRA-426 then we will be able to reproduce
any such future errors (assuming they are compaction related, rather than
memtable/BMt).
> Corrupt SSTable
> ---------------
>
> Key: CASSANDRA-452
> URL: https://issues.apache.org/jira/browse/CASSANDRA-452
> Project: Cassandra
> Issue Type: Bug
> Environment: Pre 0.4 based on r805615 on trunk w/ #370, #392, #394,
> #405, #406, #418
> Reporter: Sammy Yu
> Attachments: FriendActions-17120.tar.gz, FriendActions-17122-Data.db,
> FriendActions-17122-Filter.db, FriendActions-17122-Index.db
>
>
> We noticed on one of our node the number of SStables is growing. The
> compaction thread is alive and running. We can see that it is constantly
> trying to compact the same set of sstables. However, it is failing because
> one of the sstable is corrupt:
> ERROR [ROW-READ-STAGE:475] 2009-09-21 00:29:17,068
> DebuggableThreadPoolExecutor.java (line 125) Error in ThreadPoolExecutor
> java.lang.RuntimeException: java.io.EOFException
> at
> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:110)
> at
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:44)
> 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.readFully(RandomAccessFile.java:383)
> at java.io.RandomAccessFile.readFully(RandomAccessFile.java:361)
> at
> org.apache.cassandra.utils.FBUtilities.readByteArray(FBUtilities.java:390)
> at
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:64)
> at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:349)
> at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:309)
> at
> org.apache.cassandra.db.filter.SSTableNamesIterator.<init>(SSTableNamesIterator.java:102)
> at
> org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:69)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1467)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1401)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1420)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1401)
> at org.apache.cassandra.db.Table.getRow(Table.java:589)
> at
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65)
> at
> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:78)
> ... 4 more
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.