Stream session's index build waiting to be scheduled
Hi, Currently both secondary index build and compactions are run by compaction manager within the same thread pool (C* 2.0.x). I'm currently experiencing a situation where repair's stream sessions get stuck waiting because they don't get scheduled to build secondary indexes (and finalise the stream session) because there are already many compaction tasks pending to be finished (the node is recently bootstrapped.). Is this the expected behaviour? How is it possible to run repair on a column family with secondary indexes while one of the nodes involved in repair have loads of pending compaction tasks? Thanks, Omid
Re: Is Anti Entropy repair idempotent with respect to transferred data?
Thanks Andrey. Also found this ticket regarding this issue: https://issues.apache.org/jira/browse/CASSANDRA-2698 On Tue, Oct 16, 2012 at 8:00 PM, Andrey Ilinykh wrote: >> In my experience running repair on some counter data, the size of >> streamed data is much bigger than the cluster could possibly have lost >> messages or would be due to snapshotting at different times. >> >> I know the data will eventually be in sync on every repair, but I'm >> more interested in whether Cassandra transfers excess data and how to >> minimize this. >> >> Does any body have insights on this? >> > The problem is in granularity of Merkle tree. Cassandra sends regions > which have different hash values. It could be much bigger then a > single row. > > Andrey
Re: Nodetool repair and Leveled Compaction
I think this JIRA answers your question: https://issues.apache.org/jira/browse/CASSANDRA-2610 which in order not to duplicate work (creation of Merkle trees) repair is done on all replicas for a range. Cheers, Omid On Tue, Sep 25, 2012 at 8:27 AM, Sergey Tryuber wrote: > Hi Radim > > Unfortunately number of compaction tasks is not overestimated. The number is > decremented one-by-one and this process takes several hours for our 40GB > node(( Also, when a lot of compaction tasks appears, we see that total disk > space used (via JMX) is doubled and Cassandra really tries to compact > something. When compactions are done, "total disk space used" is back to > normal. > > > On 24 September 2012 19:04, Radim Kolar wrote: >> >> >>> Repair process by itself is going well in a background, but the issue I'm >>> concerned is a lot of unnecessary compaction tasks >> >> number in compaction tasks counter is over estimated. For example i have >> 1100 tasks left and if I will stop inserting data, all tasks will finish >> within 30 minutes. >> >> I suppose that this counter is incremented for every sstable which needs >> compaction, but its not decremented properly because you can compact about >> 20 sstables at once, and this reduces counter only by 1. > >
Re: Assertions running Cleanup on a 3-node cluster with Cassandra 1.1.4 and LCS
On Wed, Sep 12, 2012 at 9:38 AM, Janne Jalkanen wrote: > OK, so what's the worst case here? Data loss? Bad performance? Low performance is for sure a side effect. I can't comment on data loss (and I'm curious about as well) because it depends on how data off of an out-of-order sstable was being indexed and served prior to Cassandra 1.1.1 (that the bug became apparent) which is essential for counter repairs, for example. -- Omid >> The fix was released on 1.1.3 (LCS fix) and 1.1.4 (offline scrub) and >> I agree it would be helpful to have it on NEWS.txt. > > I'll file a bug on this, unless someone can get to it first :) > > /Janne
Re: is "Not a time-based UUID" serious?
On Thu, Sep 13, 2012 at 1:43 AM, Bryan Talbot wrote: > To answer my own question: yes, the error is fatal. This also means that > upgrades to 1.1.x from 1.0.x MUST use 1.0.11 or greater it seems to be > successful. > > My test upgrade from 1.0.9 to 1.1.5 left the cluster in a state that wasn't > able to come to a schema agreement and blocked schema changes. Also note that you still can't make schema changes when running with mixed 1.0.11 and 1.1.x cluster until all nodes migrated to 1.1.x. -- Omid > -Bryan > > > On Wed, Sep 12, 2012 at 2:42 PM, Bryan Talbot > wrote: >> >> I'm testing upgrading a multi-node cluster from 1.0.9 to 1.1.5 and ran >> into the error message described here: >> https://issues.apache.org/jira/browse/CASSANDRA-4195 >> >> What I can't tell is if this is a serious issue or if it can be safely >> ignored. >> >> If it is a serious issue, shouldn't the migration guides for 1.1.x require >> that upgrades cannot be rolling or that all nodes must be running 1.0.11 or >> greater first? >> >> >> 2012-09-11 17:12:46,299 [GossipStage:1] ERROR >> org.apache.cassandra.service.AbstractCassandraDaemon - Fatal exception in >> thread Thread[GossipStage:1,5,main] >> java.lang.UnsupportedOperationException: Not a time-based UUID >> at java.util.UUID.timestamp(UUID.java:308) >> at >> org.apache.cassandra.service.MigrationManager.updateHighestKnown(MigrationManager.java:121) >> at >> org.apache.cassandra.service.MigrationManager.rectify(MigrationManager.java:99) >> at >> org.apache.cassandra.service.MigrationManager.onAlive(MigrationManager.java:83) >> at org.apache.cassandra.gms.Gossiper.markAlive(Gossiper.java:806) >> at >> org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:849) >> at >> org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:908) >> at >> org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(GossipDigestAckVerbHandler.java:68) >> at >> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59) >> 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) >> >> >> -Bryan >> >
Re: Assertions running Cleanup on a 3-node cluster with Cassandra 1.1.4 and LCS
On Tue, Sep 11, 2012 at 8:33 PM, Janne Jalkanen wrote: > >> A bug in Cassandra 1.1.2 and earlier could cause out-of-order sstables >> and inter-level overlaps in CFs with Leveled Compaction. Your sstables >> generated with 1.1.3 and later should not have this issue [1] [2]. > > Does this mean that LCS on 1.0.x should be considered unsafe to > use? I'm using them for semi-wide frequently-updated CounterColumns > and they're performing much better on LCS than on STCS. That's true. "Unsafe" in the sense that your data might not be in the right shape with respect to order of keys in sstables and LCS's properties and you might need to offline-scrub when you upgrade to the latest 1.1.x. >> In case you have old Leveled-compacted sstables (generated with 1.1.2 >> or earlier. including 1.0.x) you need to run offline scrub using >> Cassandra 1.1.4 or later via /bin/sstablescrub command so it'll fix >> out-of-order sstables and inter-level overlaps caused by previous >> versions of LCS. You need to take nodes down in order to run offline >> scrub. > > The 1.1.5 README does not mention this. Should it? The fix was released on 1.1.3 (LCS fix) and 1.1.4 (offline scrub) and I agree it would be helpful to have it on NEWS.txt. Cheers, Omid > /Janne >
Re: Assertions running Cleanup on a 3-node cluster with Cassandra 1.1.4 and LCS
Could you, as Aaron suggested, open a ticket? -- Omid On Tue, Sep 11, 2012 at 2:35 PM, Rudolf van der Leeden wrote: >> Which version of Cassandra has your data been created initially with? >> A bug in Cassandra 1.1.2 and earlier could cause out-of-order sstables >> and inter-level overlaps in CFs with Leveled Compaction. Your sstables >> generated with 1.1.3 and later should not have this issue [1] [2]. >> In case you have old Leveled-compacted sstables (generated with 1.1.2 >> or earlier. including 1.0.x) you need to run offline scrub using >> Cassandra 1.1.4 or later via /bin/sstablescrub command so it'll fix >> out-of-order sstables and inter-level overlaps caused by previous >> versions of LCS. You need to take nodes down in order to run offline >> scrub. > > > The data was orginally created on a 1.1.2 cluster with STCS (i.e. NOT > leveled compaction). > After the upgrade to 1.1.4 we changed from STCS to LCS w/o problems. > Then we ran more tests and created more and very big keys with millions of > columns. > The assertion only shows up with one particular CF containing these big > keys. > So, from your explanation, I don't think an offline scrub will help. > > Thanks, > -Rudolf. >
Re: Assertions running Cleanup on a 3-node cluster with Cassandra 1.1.4 and LCS
Which version of Cassandra has your data been created initially with? A bug in Cassandra 1.1.2 and earlier could cause out-of-order sstables and inter-level overlaps in CFs with Leveled Compaction. Your sstables generated with 1.1.3 and later should not have this issue [1] [2]. In case you have old Leveled-compacted sstables (generated with 1.1.2 or earlier. including 1.0.x) you need to run offline scrub using Cassandra 1.1.4 or later via /bin/sstablescrub command so it'll fix out-of-order sstables and inter-level overlaps caused by previous versions of LCS. You need to take nodes down in order to run offline scrub. > After 3 hours the job is done and there are 11390 compaction tasks pending. > My question: Can these assertions be ignored? Or do I need to worry about > it? They can't be ignored since pending compactions elevate the upper bound on number of disk seeks you need to make to read a row and you don't get the nice guarantees of leveled compaction. Cheers, Omid [1] https://issues.apache.org/jira/browse/CASSANDRA-4411 [2] https://issues.apache.org/jira/browse/CASSANDRA-4321 On Mon, Sep 10, 2012 at 6:37 PM, Rudolf van der Leeden wrote: > Hi, > > I'm getting 5 identical assertions while running 'nodetool cleanup' on a > Cassandra 1.1.4 node with Load=104G and 80m keys. > From system.log : > > ERROR [CompactionExecutor:576] 2012-09-10 11:25:50,265 > AbstractCassandraDaemon.java (line 134) Exception in thread > Thread[CompactionExecutor:576,1,main] > java.lang.AssertionError > at > org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214) > at > org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158) > at > org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531) > at > org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254) > at > org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:992) > at > org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200) > at > org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50) > at > org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:154) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441) > at > java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) > at java.util.concurrent.FutureTask.run(FutureTask.java:138) > 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) > > After 3 hours the job is done and there are 11390 compaction tasks pending. > My question: Can these assertions be ignored? Or do I need to worry about > it? > > Thanks for your help and best regards, > -Rudolf. >
Re: Schema Disagreement after migration from 1.0.6 to 1.1.4
Do you see exceptions like "java.lang.UnsupportedOperationException: Not a time-based UUID" in log files of nodes running 1.0.6 and 1.0.9? Then it's probably due to [1] explained here [2] -- In this case you either have to upgrade all nodes to 1.1.4 or if you prefer keeping a mixed-version cluster, the 1.0.6 and 1.0.9 nodes won't be able to join the cluster again, unless you temporarily upgrade them to 1.0.11. Cheers, Omid [1] https://issues.apache.org/jira/browse/CASSANDRA-1391 [2] https://issues.apache.org/jira/browse/CASSANDRA-4195 On Wed, Sep 5, 2012 at 4:08 PM, Martin Koch wrote: > > Hi list > > We have a 5-node Cassandra cluster with a single 1.0.9 installation and four > 1.0.6 installations. > > We have tried installing 1.1.4 on one of the 1.0.6 nodes (following the > instructions on http://www.datastax.com/docs/1.1/install/upgrading). > > After bringing up 1.1.4 there are no errors in the log, but the cluster now > suffers from schema disagreement > > [default@unknown] describe cluster; > Cluster Information: >Snitch: org.apache.cassandra.locator.SimpleSnitch >Partitioner: org.apache.cassandra.dht.RandomPartitioner >Schema versions: > 59adb24e-f3cd-3e02-97f0-5b395827453f: [10.10.29.67] <- The new 1.1.4 node > > 943fc0a0-f678-11e1--339cf8a6c1bf: [10.10.87.228, 10.10.153.45, > 10.10.145.90, 10.38.127.80] <- nodes in the old cluster > > The recipe for recovering from schema disagreement > (http://wiki.apache.org/cassandra/FAQ#schema_disagreement) doesn't cover the > new directory layout. The system/Schema directory is empty save for a > snapshots subdirectory. system/schema_columnfamilies and > system/schema_keyspaces contain some files. As described in datastax's > description, we tried running nodetool upgradesstables. When this had done, > describe schema in the cli showed a schema definition which seemed correct, > but was indeed different from the schema on the other nodes in the cluster. > > Any clues on how we should proceed? > > Thanks, > /Martin Koch
Re: how is this possible ---- token range sum > 100% ?
The change is introduced in 1.1.1 [1] which takes replication factor into account [2]. That's why 18.75% * 16 = 300% (which you have replication factor of 3). [1] https://issues.apache.org/jira/browse/CASSANDRA-3412 [2] https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java#L342 On Wed, Aug 29, 2012 at 7:02 PM, Yang wrote: > I have 16 nodes, each of them should have only 6.25%, but now they all show > 18.73% > how is this possible? (this is version 1.1.2) > > thanks > Yang > > > $ nodetool -h localhost ring > Address DC RackStatus State Load > Effective-Ownership Token > > 159507359494189904748456847233641349120 > 10.28.166.83datacenter1 rack1 Down Normal 11.54 KB > 18.75% 0 > 10.40.105.247 datacenter1 rack1 Up Normal 17.23 KB > 18.75% 10633823966279326983230456482242756608 > 10.62.5.129 datacenter1 rack1 Up Normal 26.04 KB > 18.75% 21267647932558653966460912964485513216 > 10.10.215.247 datacenter1 rack1 Up Normal 31.7 KB > 18.75% 31901471898837980949691369446728269824 > 10.6.98.128 datacenter1 rack1 Up Normal 17.23 KB > 18.75% 42535295865117307932921825928971026432 > 10.190.109.13 datacenter1 rack1 Up Normal 21.66 KB > 18.75% 53169119831396634916152282411213783040 > 10.10.202.34datacenter1 rack1 Up Normal 17.23 KB > 18.75% 63802943797675961899382738893456539648 > 10.118.94.29datacenter1 rack1 Up Normal 30.7 KB > 18.75% 7443676776395522613195375699296256 > 10.28.237.220 datacenter1 rack1 Up Normal 137.04 KB > 18.75% 85070591730234615865843651857942052864 > 10.29.206.96datacenter1 rack1 Up Normal 26.04 KB > 18.75% 95704415696513942849074108340184809472 > 10.28.31.200datacenter1 rack1 Up Normal 21.62 KB > 18.75% 106338239662793269832304564822427566080 > 10.214.222.191 datacenter1 rack1 Up Normal 36.12 KB > 18.75% 116972063629072596815535021304670322688 > 10.240.38.84datacenter1 rack1 Up Normal 26.04 KB > 18.75% 127605887595351923798765477786913079296 > 10.46.141.17datacenter1 rack1 Up Normal 17.23 KB > 18.75% 138239711561631250781995934269155835904 > 10.96.69.217datacenter1 rack1 Up Normal 26.04 KB > 18.75% 148873535527910577765226390751398592512 > 10.118.119.170 datacenter1 rack1 Up Normal 17.23 KB > 18.75% 159507359494189904748456847233641349120 >
Re: Automating nodetool repair
> > Secondly, what's the need for sleep 120? > > just give the cluster a chance to settle down between repairs... > there's no real need for it, just is there "because". Actually, repair could cause unreplicated data to be streamed and new sstables to be created. New sstables could cause pending compactions and increase the potential number of sstables a row could be spread across. Therefore you might need more disk seeks to read a row and have slower read response time. If the read response time is critical, it's a good idea to wait for pending compactions to settle before repairing other neighbouring ranges that overlap replicas. -- Omid > -- > Aaron Turner > http://synfin.net/ Twitter: @synfinatic > http://tcpreplay.synfin.net/ - Pcap editing and replay tools for Unix & > Windows > Those who would give up essential Liberty, to purchase a little temporary > Safety, deserve neither Liberty nor Safety. > -- Benjamin Franklin > "carpe diem quam minimum credula postero"
Re: 1.1.3 crasch when initializing column family
It works for me on Sun's jvm. It appears to be similar to: http://www-01.ibm.com/support/docview.wss?uid=swg1IV12627 Have you tried upgrading IBM's java or using Sun's? -- Omid On Mon, Aug 13, 2012 at 1:34 PM, Robert Hellmans wrote: > ** > Hi > > I'm trying the cassandra 1.1.3 on a DualCore SLES10SP4 32bit machine. The > setup is just one single node. The cassandra receives an exception (see > below) and cores when trying to initialize a keyspace/column family. I can > reproduce the crasch with the setup/config files included in the tar.gz. > distribution. > > > 1. remove any old crap from /var/lib/cassandra > 2. start the cassandra and observe output: bin/cassandra -f > 3. In another shell, exec bin/cassandra-cli -h 127.0.0.1 -f datadef.txt > > Where datadef.txt: > ---> > create keyspace tst4 >with placement_strategy = > 'org.apache.cassandra.locator.SimpleStrategy' >and strategy_options = {replication_factor:1}; > > use tst4; > > create column family dirs_ch >with comparator = AsciiType >and column_metadata = [ >{ column_name:'str', validation_class:AsciiType}, >{ column_name:'str1', validation_class:AsciiType}, >{ column_name:'assoc_str', validation_class:AsciiType}]; > < > > > Here is (part) of the output from the when cassandra crasches: > -> > Unhandled > exception > > Type=Segmentation error > vmState=0x > J9Generic_Signal_Number=0004 Signal_Number=000b > Error_Value= Signal_Code=0001 > Handler1=B75B1443 Handler2=B758A859 > InaccessibleAddress=00B4 > EDI=B760F560 ESI=71EC1532 EAX=0022 > EBX=71D35BA0 > ECX=086050EC > EDX=0080 > > EIP=B6584E15 ES=007B DS=007B ESP=08605080 > > EFlags=00010202 CS=0073 SS=007B > EBP=082FCB00 > Module=/usr/lib/jvm/java-1.6.0-ibm-1.6.0/jre/lib/i386/libjclscar_24.so > > Module_base_address=B653F000 > Symbol=sun_misc_Unsafe_getLong__Ljava_lang_Object_2J > Symbol_address=B6584DCC > > Target=2_40_20101124_069295 (Linux > 2.6.16.60-0.85.1-bigsmp) > CPU=x86 (2 logical CPUs) (0xf4ddb000 RAM) > <--- > > > > > > If I do exactly the same but with the following file i.e just one less > column_metadata, cassandra survives. > ---> > create keyspace tst4 >with placement_strategy = > 'org.apache.cassandra.locator.SimpleStrategy' >and strategy_options = {replication_factor:1}; > > use tst4; > > create column family dirs_ch >with comparator = AsciiType >and column_metadata = [ >{ column_name:'str', validation_class:AsciiType}, >{ column_name:'assoc_str', validation_class:AsciiType}]; > <- > > > I've tried with Cassandra 1.0.9 and it is working just fine !!! > Any idea ?? > > rds /Robban > > > > > > > >
Re: Mixed cluster node with version 1.1.2 and 1.0.6 gives errors
I think the 1.0.6 nodes can't join (on a restart, for example) the ring in a mixed cluster. So you can either continue the rolling upgrade, or if you want to keep running in mixed mode (why?) you can use 1.0.11 for the older ones, but you won't be able to do schema migrations. -- Omid On Fri, Aug 3, 2012 at 1:44 AM, Roshan wrote: > > Thanks to point me the solution. So that means, I want to upgrade 1.0.6 > cluster to 1.0.11 first, then upgrade to 1.1.2 version. Is I am right? > > Thanks > > /Roshan > > > > -- > View this message in context: > http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Mixed-cluster-node-with-version-1-1-2-and-1-0-6-gives-errors-tp7581534p7581573.html > Sent from the cassandra-u...@incubator.apache.org mailing list archive at > Nabble.com.
Re: Mixed cluster node with version 1.1.2 and 1.0.6 gives errors
Hey, It is explained here: https://issues.apache.org/jira/browse/CASSANDRA-4195 -- Omid On Wed, Aug 1, 2012 at 2:39 AM, Roshan wrote: > Hi > > I have 3 node development cluster and all running 1.0.6 version without any > issue. As the part of the upgrade to 1.1.2, I just upgrade one node to > 1.1.2 > version. When start the upgraded 1.1.2 node, the other 1.0.6 nodes getting > the below exceptions? > > 2012-08-01 18:31:15,990 INFO [IncomingTcpConnection] Received connection > from newer protocol version. Ignorning > 2012-08-01 18:31:16,008 INFO [Gossiper] Node /10.1.161.202 has restarted, > now UP > 2012-08-01 18:31:16,008 INFO [Gossiper] InetAddress /10.1.161.202 is now > UP > 2012-08-01 18:31:16,010 ERROR [AbstractCassandraDaemon] Fatal exception in > thread Thread[GossipStage:1,5,main] > java.lang.UnsupportedOperationException: Not a time-based UUID > at java.util.UUID.timestamp(UUID.java:308) > at > > org.apache.cassandra.service.MigrationManager.rectify(MigrationManager.java:98) > at > > org.apache.cassandra.service.MigrationManager.onAlive(MigrationManager.java:81) > at org.apache.cassandra.gms.Gossiper.markAlive(Gossiper.java:807) > at > org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:850) > at > org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:909) > at > > org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(GossipDigestAckVerbHandler.java:68) > at > > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59) > 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) > 2012-08-01 18:31:16,013 ERROR [AbstractCassandraDaemon] Fatal exception in > thread Thread[GossipStage:1,5,main] > java.lang.UnsupportedOperationException: Not a time-based UUID > at java.util.UUID.timestamp(UUID.java:308) > at > > org.apache.cassandra.service.MigrationManager.rectify(MigrationManager.java:98) > at > > org.apache.cassandra.service.MigrationManager.onAlive(MigrationManager.java:81) > at org.apache.cassandra.gms.Gossiper.markAlive(Gossiper.java:807) > at > org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:850) > at > org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:909) > at > > org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(GossipDigestAckVerbHandler.java:68) > at > > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59) > 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) > 2012-08-01 18:31:16,383 INFO [StorageService] Node /10.1.161.202 state > jump > to normal > 2012-08-01 18:32:17,132 ERROR [AbstractCassandraDaemon] Fatal exception in > thread Thread[HintedHandoff:1,1,main] > java.lang.RuntimeException: Could not reach schema agreement with > /10.1.161.202 in 6ms > at > > org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:224) > at > > org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:239) > at > > org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:81) > at > > org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:353) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) > 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) > 2012-08-01 18:32:17,133 ERROR [AbstractCassandraDaemon] Fatal exception in > thread Thread[HintedHandoff:1,1,main] > java.lang.RuntimeException: Could not reach schema agreement with > /10.1.161.202 in 6ms > at > > org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:224) > at > > org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:239) > at > > org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:81) > at > > org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:353) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at
Dropping counter mutations taking longer than rpc_timeout
Hey, Mutations taking longer than rpc_timeout will be dropped because coordinator won't be waiting for the coordinator and will return TimeoutException to the client, if it doesn't reach the consistency level [1]. In case of counters though, since counter mutations aren't idempotent, the client is not supposed to retry an increment on TimeoutException. So why doesn't a counter mutation gets processed regardless of rpc_timeout? Cheers, Omid [1] http://wiki.apache.org/cassandra/FAQ#dropped_messages
Reduced key-cache due to memory pressure and cache size estimate
Hi, I'm trying to tune, memtable size, key cache size and heap size on Cassandra 1.1.0 but I keep having memory pressure and reduced cache size. With the following settings: heap size: 10GB (had the same issue with 8GB so I'm testing with increased heap size) memtable_total_space_in_mb: 2GB key_cache_size_in_mb: 2GB (global key cache capacity) Still, heap usage hits flush_largest_memtables_at (= 0.75) many times in a short period of time before hitting reduce_cache_sizes_at (= 0.85) that reduces the cache size and resolves memory pressure. In one instance, cache size is reported to be 1450MB before reduction and ~870MB after reduction, but the gain in heap space due to reduction in cache size is about 3GB. Could it be that the cache size estimate in megabytes isn't accurate? Thanks, Omid
Re: [RELEASE] Apache Cassandra 1.1.1 released
I think the change log for 1.1.1 is missing CASSANDRA-4150 [1]. -- Omid [1] https://issues.apache.org/jira/browse/CASSANDRA-4150 On Mon, Jun 4, 2012 at 9:52 PM, Sylvain Lebresne wrote: > The Cassandra team is pleased to announce the release of Apache Cassandra > version 1.1.1. > > Cassandra is a highly scalable second-generation distributed database, > bringing together Dynamo's fully distributed design and Bigtable's > ColumnFamily-based data model. You can read more here: > > http://cassandra.apache.org/ > > Downloads of source and binary distributions are listed in our download > section: > > http://cassandra.apache.org/download/ > > This version is the first maintenance/bug fix release[1] on the 1.1 series. As > always, please pay attention to the release notes[2] and Let us know[3] if you > were to encounter any problem. > > Enjoy! > > [1]: http://goo.gl/4Dxae (CHANGES.txt) > [2]: http://goo.gl/ZE8ZK (NEWS.txt) > [3]: https://issues.apache.org/jira/browse/CASSANDRA
Re: Cassandra 1.1.1 stack overflow on an infinite loop building IntervalTree
Thanks. Yes it's exactly the same. Will follow up there. -- Omid On Fri, Jun 8, 2012 at 5:55 PM, Sylvain Lebresne wrote: > Looks a lot like https://issues.apache.org/jira/browse/CASSANDRA-4321. > Feel free to add a comment on there if you have any additional info. > > -- > Sylvain > > On Fri, Jun 8, 2012 at 12:06 PM, Omid Aladini > wrote: > > Also looks similar to this ticket: > > > > https://issues.apache.org/jira/browse/CASSANDRA-4078 > > > > > > > > On Thu, Jun 7, 2012 at 6:48 PM, Omid Aladini > wrote: > >> > >> Hi, > >> > >> One of my 1.1.1 nodes doesn't restart due to stack overflow on building > >> the interval tree. Bumping the stack size doesn't help. Here's the stack > >> trace: > >> > >> https://gist.github.com/2889611 > >> > >> It looks more like an infinite loop on IntervalNode constructor's logic > >> than a deep tree since DEBUG log shows looping over the same intervals: > >> > >> https://gist.github.com/2889862 > >> > >> Running it with assertions enabled shows a number of sstables which the > >> first key > last key, for example: > >> > >> 2012-06-07_16:12:18.18781 java.lang.AssertionError: SSTable first key > >> DecoratedKey(2254009252149354268486114339861094, > >> 3730343137317c3438333632333932) > last key > >> DecoratedKey(22166106697727078019854024428005234814, > >> 313138323637397c3432373931353435) > >> > >> and let's the node come up without hitting IntervalNode constructor. I > >> wonder how invalid sstables get create in the first place? Is there a > way to > >> verify if other nodes in the cluster are affected as well? > >> > >> Speaking of a solution to get the node back up without wiping the data > off > >> and let it bootstrap again, I was wondering if I remove affected > sstables > >> and restart the node followed by a repair, will the node end up in a > >> consistent state? > >> > >> SStables contain counter columns and leveled compaction is used. > >> > >> Thanks, > >> Omid > > > > >
Re: Cassandra 1.1.1 stack overflow on an infinite loop building IntervalTree
Also looks similar to this ticket: https://issues.apache.org/jira/browse/CASSANDRA-4078<https://issues.apache.org/jira/browse/CASSANDRA-4078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel> On Thu, Jun 7, 2012 at 6:48 PM, Omid Aladini wrote: > Hi, > > One of my 1.1.1 nodes doesn't restart due to stack overflow on building > the interval tree. Bumping the stack size doesn't help. Here's the stack > trace: > > https://gist.github.com/2889611 > > It looks more like an infinite loop on IntervalNode constructor's logic > than a deep tree since DEBUG log shows looping over the same intervals: > > https://gist.github.com/2889862 > > Running it with assertions enabled shows a number of sstables which the > first key > last key, for example: > > 2012-06-07_16:12:18.18781 java.lang.AssertionError: SSTable first key > DecoratedKey(2254009252149354268486114339861094, > 3730343137317c3438333632333932) > last key > DecoratedKey(22166106697727078019854024428005234814, > 313138323637397c3432373931353435) > > and let's the node come up without hitting IntervalNode constructor. I > wonder how invalid sstables get create in the first place? Is there a way > to verify if other nodes in the cluster are affected as well? > > Speaking of a solution to get the node back up without wiping the data off > and let it bootstrap again, I was wondering if I remove affected sstables > and restart the node followed by a repair, will the node end up in a > consistent state? > > SStables contain counter columns and leveled compaction is used. > > Thanks, > Omid >
Cassandra 1.1.1 stack overflow on an infinite loop building IntervalTree
Hi, One of my 1.1.1 nodes doesn't restart due to stack overflow on building the interval tree. Bumping the stack size doesn't help. Here's the stack trace: https://gist.github.com/2889611 It looks more like an infinite loop on IntervalNode constructor's logic than a deep tree since DEBUG log shows looping over the same intervals: https://gist.github.com/2889862 Running it with assertions enabled shows a number of sstables which the first key > last key, for example: 2012-06-07_16:12:18.18781 java.lang.AssertionError: SSTable first key DecoratedKey(2254009252149354268486114339861094, 3730343137317c3438333632333932) > last key DecoratedKey(22166106697727078019854024428005234814, 313138323637397c3432373931353435) and let's the node come up without hitting IntervalNode constructor. I wonder how invalid sstables get create in the first place? Is there a way to verify if other nodes in the cluster are affected as well? Speaking of a solution to get the node back up without wiping the data off and let it bootstrap again, I was wondering if I remove affected sstables and restart the node followed by a repair, will the node end up in a consistent state? SStables contain counter columns and leveled compaction is used. Thanks, Omid
Re: Losing key cache on restart
Hey, Sorry for the late response. On Wed, Apr 25, 2012 at 1:36 AM, aaron morton wrote: > - Cassandra log reports 12,955,585 of them have been saved on the last save > events. > > Has their been much activity between saves ? For testing I set the key_cache_save_period to a short period of 10 minutes. How would this affect the result? Does saving the cache also prunes duplicate in-memory elements? > Nothing jumps out. There is a setting for the max entries to store, but this > only applies to the row cache. Can you reproduce issue in a dev environment > ? So far I haven't been able to reproduce this in development environment. > When running the key cache holds keys of the form so > there is an entry for each SSTable the key appears in. When saved only the > DecoratedKey's are stored, and the key cache is rebuilt on startup when > iterating over the index files. e.g. ig you have 12 entries in the keycache, > it may only be 4 unique keys and that is all that is written when saving the > cache. If you have 12 entries for the same key on 12 sstables, this means your data is spread across 12 sstables, how could cassandra deduplicate them to 4? I'll try to reproduce/debug it as well. Thanks, Omid > > From a quick look at the code it looks like the code is writing all > DecoratedKeys , not just the unique ones. This may be mucking up the > reported numbers, I'll take a look later. > > If you can reproduce it simply it would help. > > Cheers > > - > Aaron Morton > Freelance Developer > @aaronmorton > http://www.thelastpickle.com > > On 25/04/2012, at 1:11 AM, Omid Aladini wrote: > > Hi, > > I'm experiencing losing a part of key cache on restart on Cassandra 1.0.7. > For example: > > - cfstats reports key cache size of 13,040,502 with capacity of 15,000,000. > - Cassandra log reports 12,955,585 of them have been saved on the last save > events. > - On restart Cassandra reads saved cache. > - cfstats reports key cache size of only 2,833,586 with correct capacity of > 15,000,000. > > There is no sign that the cache size is reduced due to memory pressure. The > key cache capacity is set manually via cassandra-cli. > > Has anyone else encountered this problem or is it a known issue? > > Thanks, > Omid > >
Losing key cache on restart
Hi, I'm experiencing losing a part of key cache on restart on Cassandra 1.0.7. For example: - cfstats reports key cache size of 13,040,502 with capacity of 15,000,000. - Cassandra log reports 12,955,585 of them have been saved on the last save events. - On restart Cassandra reads saved cache. - cfstats reports key cache size of only 2,833,586 with correct capacity of 15,000,000. There is no sign that the cache size is reduced due to memory pressure. The key cache capacity is set manually via cassandra-cli. Has anyone else encountered this problem or is it a known issue? Thanks, Omid
Off-heap row cache and mmapped sstables
Hi, Cassandra issues an mlockall [1] before mmap-ing sstables to prevent the kernel from paging out heap space in favor of memory-mapped sstables. I was wondering, what happens to the off-heap row cache (saved or unsaved)? Is it possible that the kernel pages out off-heap row cache in favor of resident mmap-ed sstable pages? Thanks, Omid [1] http://pubs.opengroup.org/onlinepubs/007908799/xsh/mlockall.html
Re: Resident size growth
Thanks. Yes it's due to mmappd SSTables pages that count as resident size. Jeremiah: mmap isn't through JNA, it's via java.nio.MappedByteBuffer I think. -- Omid On Mon, Apr 9, 2012 at 4:15 PM, Jeremiah Jordan wrote: > He says he disabled JNA. You can't mmap without JNA can you? > > On Apr 9, 2012, at 4:52 AM, aaron morton wrote: > > see http://wiki.apache.org/cassandra/FAQ#mmap > > Cheers > > - > Aaron Morton > Freelance Developer > @aaronmorton > http://www.thelastpickle.com > > On 9/04/2012, at 5:09 AM, ruslan usifov wrote: > > mmap sstables? It's normal > > 2012/4/5 Omid Aladini >> >> Hi, >> >> I'm experiencing a steady growth in resident size of JVM running >> Cassandra 1.0.7. I disabled JNA and off-heap row cache, tested with >> and without mlockall disabling paging, and upgraded to JRE 1.6.0_31 to >> prevent this bug [1] to leak memory. Still JVM's resident set size >> grows steadily. A process with Xmx=2048M has grown to 6GB resident >> size and one with Xmx=8192M to 16GB in a few hours and increasing. Has >> anyone experienced this? Any idea how to deal with this issue? >> >> Thanks, >> Omid >> >> [1] http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=7066129 > > > >
Resident size growth
Hi, I'm experiencing a steady growth in resident size of JVM running Cassandra 1.0.7. I disabled JNA and off-heap row cache, tested with and without mlockall disabling paging, and upgraded to JRE 1.6.0_31 to prevent this bug [1] to leak memory. Still JVM's resident set size grows steadily. A process with Xmx=2048M has grown to 6GB resident size and one with Xmx=8192M to 16GB in a few hours and increasing. Has anyone experienced this? Any idea how to deal with this issue? Thanks, Omid [1] http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=7066129