You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Ben Coverston <be...@datastax.com> on 2011/02/07 21:52:13 UTC

Re: Argh: Data Corruption (LOST DATA) (0.7.0)

Dan,

Do you have any more information on this issue? Have you been able to 
discover anything from exporing your SSTables to JSON?

Thanks,
Ben

On 1/29/11 12:45 PM, Dan Hendry wrote:
>
> I am once again having severe problems with my Cassandra cluster. This 
> time, I straight up cannot read sections of data (consistency level 
> ONE). Client side, I am seeing timeout exceptions. On the Cassandra 
> node, I am seeing errors as shown below. I don't understand what has 
> happened or how to fix it. I also don't understand how I am seeing 
> errors on only one node, using consistency level ONE with a rf=2 and 
> yet clients are failing. I have tried turning on debug logging but 
> that been no help, the logs roll over (20 mb) in < 10 seconds (the 
> cluster is being used quite heavily).
>
> My cluster has been working fine for weeks the suddenly, I had a 
> corrupt SSTable which caused me all sorts of grief (outlined in 
> pervious emails). I was able to solve the problem by turning down the 
> max compaction threshold then figuring out which SSTable was corrupt 
> by watching which minor compactions failed. After that, I straight up 
> deleted the on-disk data. Now I am having problems on a different node 
> (but adjacent in the ring) for what I am almost certain is the same 
> column family (presumably the same row/column). At this point, the 
> data is effectively lost as I know 1 of the 2 replicas was completely 
> deleted.
>
> Is there any advice going forward? My next course of action was going 
> to be exporting all of the sstables to JSON using the provided tool 
> and trying to look it over manually to see what the problem actually 
> is (if exporting will even work). I am not sure how useful this will 
> be as there is nearly 80 GB of data for this CF on a single node. What 
> is more concerning is that I have no idea how this problem initially 
> popped up. I have performed hardware tests and nothing seems to be 
> malfunctioning. Furthermore, the fact that these issues have 'jumped' 
> nodes is a strong indication to me this is a Cassandra problem.
>
> There is a Cassandra bug here somewhere, if only in the way corrupt 
> columns are dealt with.
>
> db (85098417 bytes)
>
> ERROR [ReadStage:221] 2011-01-29 12:42:39,153 
> DebuggableThreadPoolExecutor.java (line 103) Error in ThreadPoolExecutor
>
> java.lang.RuntimeException: java.io.IOException: Invalid 
> localDeleteTime read: -1516572672
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:124)
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:47)
>
>         at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136)
>
>         at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131)
>
>         at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
>
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
>
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326)
>
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230)
>
>         at 
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:68)
>
>         at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136)
>
>         at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131)
>
>         at 
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:118)
>
>         at 
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:142)
>
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1230)
>
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1107)
>
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1077)
>
>         at org.apache.cassandra.db.Table.getRow(Table.java:384)
>
>         at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63)
>
>         at 
> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:68)
>
>         at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:63)
>
>         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.IOException: Invalid localDeleteTime read: -1516572672
>
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:356)
>
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:313)
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetcher.getNextBlock(IndexedSliceReader.java:180)
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:119)
>
>         ... 22 more
>
> ERROR [ReadStage:210] 2011-01-29 12:42:41,529 
> DebuggableThreadPoolExecutor.java (line 103) Error in ThreadPoolExecutor
>
> java.lang.RuntimeException: 
> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: 
> invalid column name length 0
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:124)
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:47)
>
>         at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136)
>
>         at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131)
>
>         at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
>
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
>
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326)
>
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230)
>
>         at 
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:68)
>
>         at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136)
>
>         at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131)
>
>         at 
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:118)
>
>         at 
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:142)
>
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1230)
>
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1107)
>
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1077)
>
>         at org.apache.cassandra.db.Table.getRow(Table.java:384)
>
>         at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63)
>
>         at 
> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:68)
>
>         at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:63)
>
>         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: 
> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: 
> invalid column name length 0
>
>         at 
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:68)
>
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:364)
>
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:313)
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetcher.getNextBlock(IndexedSliceReader.java:180)
>
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:119)
>
>         ... 22 more
>
> Dan Hendry
>
> (403) 660-2297
>

Re: Argh: Data Corruption (LOST DATA) (0.7.0)

Posted by Terje Marthinussen <tm...@gmail.com>.
Hi,

Unfortunately, this patch is already included in the build I have.

Thanks for the suggestion though!
Terje

On Sat, Mar 5, 2011 at 7:47 PM, Sylvain Lebresne <sy...@datastax.com>wrote:

> Also, if you can, please be sure to try the new 0.7.3 release. We had a bug
> with the compaction of superColumns for instance that is fixed there (
> https://issues.apache.org/jira/browse/CASSANDRA-2104). It also ships with
> a new scrub command that tries to find if your sstables are corrupted and
> repair them in that event. I can be worth trying it too.
>
> --
> Sylvain
>
>
> On Fri, Mar 4, 2011 at 7:04 PM, Benjamin Coverston <
> ben.coverston@datastax.com> wrote:
>
>>  Hi Terje,
>>
>> Can you attach the portion of your logs that shows the exceptions
>> indicating corruption? Which version are you on right now?
>>
>> Ben
>>
>>
>> On 3/4/11 10:42 AM, Terje Marthinussen wrote:
>>
>> We are seeing various other messages as well related to deserialization,
>> so this seems to be some random corruption somewhere, but so far it may seem
>> to be limited to supercolumns.
>>
>>  Terje
>>
>>  On Sat, Mar 5, 2011 at 2:26 AM, Terje Marthinussen <
>> tmarthinussen@gmail.com> wrote:
>>
>>> Hi,
>>>
>>>  Did you get anywhere on this problem?
>>>
>>>  I am seeing similar errors unfortunately :(
>>>
>>>  I tried to add some quick error checking to the serialization, and it
>>> seems like the data is ok there.
>>>
>>>  Some indication that this occurs in compaction and maybe in hinted
>>> handoff, but no indication that it occurs for sstables with newly written
>>> data.
>>>
>>>  Terje
>>>
>>
>>
>> --
>> Ben Coverston
>> DataStax -- The Apache Cassandra Companyhttp://www.datastax.com/
>>
>>
>

Re: Argh: Data Corruption (LOST DATA) (0.7.0)

Posted by Sylvain Lebresne <sy...@datastax.com>.
Also, if you can, please be sure to try the new 0.7.3 release. We had a bug
with the compaction of superColumns for instance that is fixed there (
https://issues.apache.org/jira/browse/CASSANDRA-2104). It also ships with a
new scrub command that tries to find if your sstables are corrupted and
repair them in that event. I can be worth trying it too.

--
Sylvain

On Fri, Mar 4, 2011 at 7:04 PM, Benjamin Coverston <
ben.coverston@datastax.com> wrote:

>  Hi Terje,
>
> Can you attach the portion of your logs that shows the exceptions
> indicating corruption? Which version are you on right now?
>
> Ben
>
>
> On 3/4/11 10:42 AM, Terje Marthinussen wrote:
>
> We are seeing various other messages as well related to deserialization, so
> this seems to be some random corruption somewhere, but so far it may seem to
> be limited to supercolumns.
>
>  Terje
>
>  On Sat, Mar 5, 2011 at 2:26 AM, Terje Marthinussen <
> tmarthinussen@gmail.com> wrote:
>
>> Hi,
>>
>>  Did you get anywhere on this problem?
>>
>>  I am seeing similar errors unfortunately :(
>>
>>  I tried to add some quick error checking to the serialization, and it
>> seems like the data is ok there.
>>
>>  Some indication that this occurs in compaction and maybe in hinted
>> handoff, but no indication that it occurs for sstables with newly written
>> data.
>>
>>  Terje
>>
>
>
> --
> Ben Coverston
> DataStax -- The Apache Cassandra Companyhttp://www.datastax.com/
>
>

Re: Argh: Data Corruption (LOST DATA) (0.7.0)

Posted by Benjamin Coverston <be...@datastax.com>.
Hi Terje,

Can you attach the portion of your logs that shows the exceptions 
indicating corruption? Which version are you on right now?

Ben

On 3/4/11 10:42 AM, Terje Marthinussen wrote:
> We are seeing various other messages as well related to 
> deserialization, so this seems to be some random corruption somewhere, 
> but so far it may seem to be limited to supercolumns.
>
> Terje
>
> On Sat, Mar 5, 2011 at 2:26 AM, Terje Marthinussen 
> <tmarthinussen@gmail.com <ma...@gmail.com>> wrote:
>
>     Hi,
>
>     Did you get anywhere on this problem?
>
>     I am seeing similar errors unfortunately :(
>
>     I tried to add some quick error checking to the serialization, and
>     it seems like the data is ok there.
>
>     Some indication that this occurs in compaction and maybe in hinted
>     handoff, but no indication that it occurs for sstables with newly
>     written data.
>
>     Terje
>
>

-- 
Ben Coverston
DataStax -- The Apache Cassandra Company
http://www.datastax.com/


Re: Argh: Data Corruption (LOST DATA) (0.7.0)

Posted by Terje Marthinussen <tm...@gmail.com>.
We are seeing various other messages as well related to deserialization, so
this seems to be some random corruption somewhere, but so far it may seem to
be limited to supercolumns.

Terje

On Sat, Mar 5, 2011 at 2:26 AM, Terje Marthinussen
<tm...@gmail.com>wrote:

> Hi,
>
> Did you get anywhere on this problem?
>
> I am seeing similar errors unfortunately :(
>
> I tried to add some quick error checking to the serialization, and it seems
> like the data is ok there.
>
> Some indication that this occurs in compaction and maybe in hinted handoff,
> but no indication that it occurs for sstables with newly written data.
>
> Terje
>

Re: Argh: Data Corruption (LOST DATA) (0.7.0)

Posted by Terje Marthinussen <tm...@gmail.com>.
Hi,

Did you get anywhere on this problem?

I am seeing similar errors unfortunately :(

I tried to add some quick error checking to the serialization, and it seems
like the data is ok there.

Some indication that this occurs in compaction and maybe in hinted handoff,
but no indication that it occurs for sstables with newly written data.

Terje