incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Aaron Morton <aa...@thelastpickle.com>
Subject Re: MemtablePostFlusher pending
Date Wed, 23 Oct 2013 20:54:35 GMT
On a plane and cannot check jira but…

> ERROR [FlushWriter:216] 2013-10-07 07:11:46,538 CassandraDaemon.java (line 186) Exception
in thread Thread[FlushWriter:216,5,main]
> java.lang.AssertionError
>         at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:198)
Happened because we tried to write a row to disk that had zero columns and was not a row level
tombstone. 


> ERROR [ValidationExecutor:2] 2013-10-23 08:39:27,558 CassandraDaemon.java (line 185)
Exception in thread Thread[ValidationExecutor:2,1,main]
> java.lang.AssertionError
>         at org.apache.cassandra.db.compaction.PrecompactedRow.update(PrecompactedRow.java:171)
>         at org.apache.cassandra.repair.Validator.rowHash(Validator.java:198)
>         at org.apache.cassandra.repair.Validator.add(Validator.java:151)

I *think* this is happening for similar reasons. 

(notes to self below)…

    public PrecompactedRow(CompactionController controller, List<SSTableIdentityIterator>
rows)
    {
        this(rows.get(0).getKey(),
             removeDeletedAndOldShards(rows.get(0).getKey(), controller, merge(rows, controller)));
    }

 results in call to this on CFS

    public static ColumnFamily removeDeletedCF(ColumnFamily cf, int gcBefore)
    {
        cf.maybeResetDeletionTimes(gcBefore);
        return cf.getColumnCount() == 0 && !cf.isMarkedForDelete() ? null : cf;
    }

If the CF has zero columns and is not marked for delete the CF will be null, and the PreCompacedRow
will be created with a non cf. This is the source of the assertion. 

Any information on how you are using cassandra, does the zero columns no row delete idea sound
like something you are doing ? 


This may already be fixed. Will take a look later when on the ground. 

Cheers


-----------------
Aaron Morton
New Zealand
@aaronmorton

Co-Founder & Principal Consultant
Apache Cassandra Consulting
http://www.thelastpickle.com

On 23/10/2013, at 9:50 PM, Kais Ahmed <kais@neteck-fr.com> wrote:

> Thanks robert,
> 
> For info if it helps to fix the bug i'm starting the downgrade, i restart all the node
and do a repair and there are a lot of error like this :
> 
> EERROR [ValidationExecutor:2] 2013-10-23 08:39:27,558 Validator.java (line 242) Failed
creating a merkle tree for [repair #9f9b7fc0-3bbe-11e3-a220-b18f7c69b044 on ks01/messages,
(8746393670077301406,8763948586274310360]], /172.31.38.135 (see log for details)
> ERROR [ValidationExecutor:2] 2013-10-23 08:39:27,558 CassandraDaemon.java (line 185)
Exception in thread Thread[ValidationExecutor:2,1,main]
> java.lang.AssertionError
>         at org.apache.cassandra.db.compaction.PrecompactedRow.update(PrecompactedRow.java:171)
>         at org.apache.cassandra.repair.Validator.rowHash(Validator.java:198)
>         at org.apache.cassandra.repair.Validator.add(Validator.java:151)
>         at org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:798)
>         at org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:60)
>         at org.apache.cassandra.db.compaction.CompactionManager$8.call(CompactionManager.java:395)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:724)
> 
> And the repair stop after this error :
> 
> ERROR [FlushWriter:9] 2013-10-23 08:39:32,979 CassandraDaemon.java (line 185) Exception
in thread Thread[FlushWriter:9,5,main]
> java.lang.AssertionError
>         at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:198)
>         at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:186)
>         at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:358)
>         at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:317)
>         at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:724)
> 
> 
> 
> 
> 
> 
> 
> 
> 2013/10/22 Robert Coli <rcoli@eventbrite.com>
> On Mon, Oct 21, 2013 at 11:57 PM, Kais Ahmed <kais@neteck-fr.com> wrote:
> I will try to create a new cluster 1.2 and copy data, can you tell me please the best
pratice to do this, do i have to use sstable2json / json2sstable or other method.
> 
> Unfortunately to downgrade versions you are going to need to use a method like sstable2json/json2sstable.
> 
> Other bulkload options, which mostly don't apply in the downgrade case, here :
> 
> http://www.palominodb.com/blog/2012/09/25/bulk-loading-options-cassandra
> 
> =Rob
> 


Mime
View raw message