incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ravi prasad <raviprasad...@yahoo.com>
Subject Re: exception during add node due to test beforeAppend on SSTableWriter
Date Thu, 06 Feb 2014 19:19:23 GMT
I'm seeing the same with cassandra-2.0.4 during compaction, after lot of sstable files are
streamed after bootstrap/repair. Strange thing is, the 'Last written key >= current key'
exception during compaction of L0, L1 sstables, goes away after restarting cassandra. But,
then see those warnings about overlapping sstables. 

I think this change in https://issues.apache.org/jira/browse/CASSANDRA-5921 is causing overlapping
of sstables in L1. Didn't used to see this with cassandra-1.2.9 which had https://issues.apache.org/jira/browse/CASSANDRA-5907
fixed.  Can you open a jira reporting this issue?





On Thursday, February 6, 2014 4:31 AM, "Desimpel, Ignace" <Ignace.Desimpel@nuance.com>
wrote:
 
 
Also, these nodes and data are entirely created by a 2.0.4 code, so should not really be a
1.1.x related bug.
Also, I restarted the whole test, thus completely new database, and I get similar problems.
 
From:Desimpel, Ignace 
Sent: vrijdag 31 januari 2014 18:02
To: user@cassandra.apache.org
Subject: exception during add node due to test beforeAppend on SSTableWriter
 
The join with auto bootstrap itself was finished. So I restarted the added node. During restart
I saw a message indicating that something is wrong about this row and sstable.
Of course, in my case I did not drop sstable from another node. But I did decommission and
add the node, so that is still a kind of ‘data-from-another-node’.
 
At level 2, SSTableReader(path='../../../../data/cdi.cassandra.cdi/dbdatafile/Ks100K/ForwardStringFunction/Ks100K-ForwardStringFunction-jb-67-Data.db')
[DecoratedKey(065864ce01024e4e505300, 065864ce01024e4e505300), DecoratedKey(14c9d35e0102646973706f736974696f6e7300,
14c9d35e0102646973706f736974696f6e7300)] overlaps SSTableReader(path='../../../../data/cdi.cassandra.cdi/dbdatafile/Ks100K/ForwardStringFunction/Ks100K-ForwardStringFunction-jb-64-Data.db')
[DecoratedKey(068c2e4101024d6f64616c207665726200, 068c2e4101024d6f64616c207665726200), DecoratedKey(06c566b4010244657465726d696e657200,
06c566b4010244657465726d696e657200)].  This could be caused by a bug in Cassandra 1.1.0 ..
1.1.3 or due to the fact that you have dropped sstables from another node into the data directory.
Sending back to L0.  If you didn't drop in sstables, and have not yet run scrub, you should
do so since you may also have rows out-of-order within an sstable
 
 
 
From:Desimpel, Ignace 
Sent: vrijdag 31 januari 2014 17:43
To: user@cassandra.apache.org
Subject: exception during add node due to test beforeAppend on SSTableWriter
 
4 node, byte ordered, LCS, 3 Compaction Executors, replication factor 1
Code is 2.0.4 version but with patch for CASSANDRA-6638 However, no cleanup is run so patch
should not play a roll
 
4 node cluster is started and insert/queries are done up to about only 10 GB of data on each
node.
Then decommission one node, and delete local files.
Then add node again.
Exception : see below.
 
Any idea?
 
Regards,
Ignace Desimpel
 
	* 2014-01-31 17:12:02.600 ==>> Bootstrap is streaming data from other nodes... Please
wait ... 
	* 2014-01-31 17:12:02.600 ==>> Bootstrap stream state : rx= 29.00 tx= 100.00 Please
wait ... 
	* 2014-01-31 17:12:18.908 Enqueuing flush of Memtable-compactions_in_progress@350895652(0/0
serialized/live bytes, 1 ops) 
	* 2014-01-31 17:12:18.908 Writing Memtable-compactions_in_progress@350895652(0/0 serialized/live
bytes, 1 ops) 
	* 2014-01-31 17:12:19.009 Completed flushing ../../../../data/cdi.cassandra.cdi/dbdatafile/system/compactions_in_progress/system-compactions_in_progress-jb-74-Data.db
(42 bytes) for commitlog position ReplayPosition(segmentId=1391184546183, position=561494)

	* 2014-01-31 17:12:19.018 Exception in thread Thread[CompactionExecutor:1,1,main] 
	* java.lang.RuntimeException: Last written key DecoratedKey(8afc923701038000000000178575,
8afc923701038000000000178575) >= current key DecoratedKey(6e0bb95501038000000003dfdd1d,
6e0bb95501038000000003dfdd1d) writing into /media/datadrive1/cdi.cassandra.cdi/dbdatafile/Ks100K/ForwardLongFunction/Ks100K-ForwardLongFunction-tmp-jb-159-Data.db

	* at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142) ~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT]

	* at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165) ~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT]

	* at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160) ~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT]

	* at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT] 
	* at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT]

	* at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT] 
	* at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT] 
	* at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
~[apache-cassandra-2.0.4-SNAPSHOT.jar:2.0.4-SNAPSHOT] 
	* at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_40]

	* at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_40] 
	* at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_40]

	* at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_40]

	* at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40] 
Mime
View raw message