Re: C* 2.1-rc2 gets unstable after a 'DROP KEYSPACE' command ?
Hello, After a 'DROP TABLE' command that returns errors={}, last_host=127.0.0.1 (like most DROP commands do) from CQLSH with C* 2.1.0-rc2, I stopped C*. And I can not start one node. It says : ERROR 09:18:34 Exception encountered during startup java.lang.NullPointerException: null at org.apache.cassandra.db.Directories.init(Directories.java:191) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:553) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:245) [apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:455) [apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:544) [apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] java.lang.NullPointerException at org.apache.cassandra.db.Directories.init(Directories.java:191) at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:553) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:245) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:455) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:544) Exception encountered during startup: null I do not now if it can help. Fabrice LARCHER 2014-07-18 7:23 GMT+02:00 Fabrice Larcher fabrice.larc...@level5.fr: Hello, I still experience a similar issue after a 'DROP KEYSPACE' command with C* 2.1-rc3. Connection to the node may fail after a 'DROP'. But I did not see this issue with 2.1-rc1 (- it seems like to be a regression brought with 2.1-rc2). Fabrice LARCHER 2014-07-17 9:19 GMT+02:00 Benedict Elliott Smith belliottsm...@datastax.com: Also https://issues.apache.org/jira/browse/CASSANDRA-7437 and https://issues.apache.org/jira/browse/CASSANDRA-7465 for rc3, although the CounterCacheKey assertion looks like an independent (though comparatively benign) bug I will file a ticket for. Can you try this against rc3 to see if the problem persists? You may see the last exception, but it shouldn't affect the stability of the cluster. If either of the other exceptions persist, please file a ticket. On Thu, Jul 17, 2014 at 1:41 AM, Tyler Hobbs ty...@datastax.com wrote: This looks like https://issues.apache.org/jira/browse/CASSANDRA-6959, but that was fixed for 2.1.0-rc1. Is there any chance you can put together a script to reproduce the issue? On Thu, Jul 10, 2014 at 8:51 AM, Pavel Kogan pavel.ko...@cortica.com wrote: It seems that memtable tries to flush itself to SSTable of not existing keyspace. I don't know why it is happens, but probably running nodetool flush before drop should prevent this issue. Pavel On Thu, Jul 10, 2014 at 4:09 AM, Fabrice Larcher fabrice.larc...@level5.fr wrote: Hello, I am using the 'development' version 2.1-rc2. With one node (=localhost), I get timeouts trying to connect to C* after running a 'DROP KEYSPACE' command. I have following error messages in system.log : INFO [SharedPool-Worker-3] 2014-07-09 16:29:36,578 MigrationManager.java:319 - Drop Keyspace 'test_main' (...) ERROR [MemtableFlushWriter:6] 2014-07-09 16:29:37,178 CassandraDaemon.java:166 - Exception in thread Thread[MemtableFlushWriter:6,5,main] java.lang.RuntimeException: Last written key DecoratedKey(91e7f660-076f-11e4-a36d-28d2444c0b1b, 52446dde90244ca49789b41671e4ca7c) = current key DecoratedKey(91e7f660-076f-11e4-a36d-28d2444c0b1b, 52446dde90244ca49789b41671e4ca7c) writing into ./../data/data/test_main/user-911d5360076f11e4812d3d4ba97474ac/test_main-user.user_account-tmp-ka-1-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:215) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:351) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:314) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.jar:na] at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1054) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at
Re: C* 2.1-rc2 gets unstable after a 'DROP KEYSPACE' command ?
It would be nice if you can try with 2.1.0-rc5 (there has been quite a bit of bug fixes since rc2). If you can still reproduce that NPE there, please do open a jira ticket with the reproduction steps. On Thu, Aug 7, 2014 at 11:29 AM, Fabrice Larcher fabrice.larc...@level5.fr wrote: Hello, After a 'DROP TABLE' command that returns errors={}, last_host=127.0.0.1 (like most DROP commands do) from CQLSH with C* 2.1.0-rc2, I stopped C*. And I can not start one node. It says : ERROR 09:18:34 Exception encountered during startup java.lang.NullPointerException: null at org.apache.cassandra.db.Directories.init(Directories.java:191) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:553) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:245) [apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:455) [apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:544) [apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] java.lang.NullPointerException at org.apache.cassandra.db.Directories.init(Directories.java:191) at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:553) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:245) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:455) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:544) Exception encountered during startup: null I do not now if it can help. Fabrice LARCHER 2014-07-18 7:23 GMT+02:00 Fabrice Larcher fabrice.larc...@level5.fr: Hello, I still experience a similar issue after a 'DROP KEYSPACE' command with C* 2.1-rc3. Connection to the node may fail after a 'DROP'. But I did not see this issue with 2.1-rc1 (- it seems like to be a regression brought with 2.1-rc2). Fabrice LARCHER 2014-07-17 9:19 GMT+02:00 Benedict Elliott Smith belliottsm...@datastax.com: Also https://issues.apache.org/jira/browse/CASSANDRA-7437 and https://issues.apache.org/jira/browse/CASSANDRA-7465 for rc3, although the CounterCacheKey assertion looks like an independent (though comparatively benign) bug I will file a ticket for. Can you try this against rc3 to see if the problem persists? You may see the last exception, but it shouldn't affect the stability of the cluster. If either of the other exceptions persist, please file a ticket. On Thu, Jul 17, 2014 at 1:41 AM, Tyler Hobbs ty...@datastax.com wrote: This looks like https://issues.apache.org/jira/browse/CASSANDRA-6959, but that was fixed for 2.1.0-rc1. Is there any chance you can put together a script to reproduce the issue? On Thu, Jul 10, 2014 at 8:51 AM, Pavel Kogan pavel.ko...@cortica.com wrote: It seems that memtable tries to flush itself to SSTable of not existing keyspace. I don't know why it is happens, but probably running nodetool flush before drop should prevent this issue. Pavel On Thu, Jul 10, 2014 at 4:09 AM, Fabrice Larcher fabrice.larc...@level5.fr wrote: Hello, I am using the 'development' version 2.1-rc2. With one node (=localhost), I get timeouts trying to connect to C* after running a 'DROP KEYSPACE' command. I have following error messages in system.log : INFO [SharedPool-Worker-3] 2014-07-09 16:29:36,578 MigrationManager.java:319 - Drop Keyspace 'test_main' (...) ERROR [MemtableFlushWriter:6] 2014-07-09 16:29:37,178 CassandraDaemon.java:166 - Exception in thread Thread[MemtableFlushWriter:6,5,main] java.lang.RuntimeException: Last written key DecoratedKey(91e7f660-076f-11e4-a36d-28d2444c0b1b, 52446dde90244ca49789b41671e4ca7c) = current key DecoratedKey(91e7f660-076f-11e4-a36d-28d2444c0b1b, 52446dde90244ca49789b41671e4ca7c) writing into ./../data/data/test_main/user-911d5360076f11e4812d3d4ba97474ac/test_main-user.user_account-tmp-ka-1-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:215) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:351) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:314) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.0-rc2.jar:2.1.0-rc2]
Decommissioning a datacenter deletes the data (on decommissioned datacenter)
tl;dr: Decommissioning datacenters by running nodetool decommission on a node deletes the data on the decommissioned node - is this expected ? I am trying our some tests on my multi-datacenter setup. Somewhere in the docs I read that decommissioning a node will stream its data to other nodes but it still retains its copy of the data. I was expecting the same behavior with multiple datacenters. I am using cassandra 1.2.12. Following are my observations: Lets say I have a datacenter DC1 which has keyspace keyspace_dc_1 and I have another datacenter DC2 which has keyspace keyspace_dc_2. They already have some data in them. I add DC2 to DC1, update the replication factors on both the keyspaces. Looking at the gossipinfo, I can see that the schemas are synced. I then look at the cfstats output and I can see then both the keyspaces are replicated on both the datacenters (also on the disk, as I can see a non-zero sstable count). Now, I decommission DC2: 1) Update the replication factors for the keyspaces. 2) Run nodetool decommission on all the nodes. I see that I have lost all my keyspaces (and data), the keyspaces from DC1 and DC2. This does not seem normal to me, is this expected ? Thanks, Sandeep
Delete By Partition Key Implementation
Hello all Usually, when using DELETE in CQL3 on some fields, C* creates tombstone columns for those fields. Now if I delete a whole PARTITION (delete from MyTable where partitionKey=...), what will C* do ? Will it create as many tombstones as there are physical columns on this partition or will it just mark this partition as deleted (Row Key deletion marker) ? On a side note, if I insert a bunch of physical columns in one partition with the SAME ttl value, after a while they will appear as expired, would C* need to scan the whole partition on disk to see which columns to expire or could it see that the whole partition is indeed expired thanks to meta data/ Partition key cache kept in memory ? I was thinking about the estimate histograms for TTL but I don't know in detail how it work Regards Duy Hai DOAN
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
On Thu, Aug 7, 2014 at 8:26 AM, srmore comom...@gmail.com wrote: tl;dr: Decommissioning datacenters by running nodetool decommission on a node deletes the data on the decommissioned node - is this expected ? What does deletes mean? What does lost all my keyspaces (and data) mean? =Rob
Re: NPE in UUIDGen.decompose - Cassandra 1.2.9
On Tue, Aug 5, 2014 at 6:01 PM, Elias Ross gen...@noderunner.net wrote: Has this been fixed? Is this a new bug? This is Cassandra 1.2.9. Dunno, though as you say there are Decommission improvements upstream in 2.x. If I were you, and could repro this, I would : 1) file a JIRA with repro steps 2) upgrade to 1.2.18 3) see if I could still repro =Rob
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
Hello Rob Sorry for being ambiguous. By deletes I mean that running decommission I can no longer see any keyspaces owned by this node or replicated by other nodes using the cfstats command. I am also seeing the same behavior when I remove a single node from a cluster (without datacenters). On Thu, Aug 7, 2014 at 11:43 AM, Robert Coli rc...@eventbrite.com wrote: On Thu, Aug 7, 2014 at 8:26 AM, srmore comom...@gmail.com wrote: tl;dr: Decommissioning datacenters by running nodetool decommission on a node deletes the data on the decommissioned node - is this expected ? What does deletes mean? What does lost all my keyspaces (and data) mean? =Rob
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
On Thu, Aug 7, 2014 at 10:04 AM, srmore comom...@gmail.com wrote: Sorry for being ambiguous. By deletes I mean that running decommission I can no longer see any keyspaces owned by this node or replicated by other nodes using the cfstats command. I am also seeing the same behavior when I remove a single node from a cluster (without datacenters). I'm still not fully parsing you, but clusters should never forget schema as a result of decommission. Is that what you are saying is happening? (In fact, even the decommissioned node itself does not forget its schema, which I personally consider a bug.) =Rob
Re: Migration from Cassandra 1.2.5 to Cassandra 2.0.8 with changed partitioner settings
Did the jmx path work? On Thu, Jul 31, 2014 at 11:16 AM, thorsten.s...@t-systems.com wrote: Well, we ran StorageService.bulkLoad via JMX. According to http://www.datastax.com/dev/blog/bulk-loading this should have the same effect and can be done on the same machine: Because the sstableloader uses gossip to communicate with other nodes, if launched on the same machine that a given Cassandra node, it will need to use a different network interface than the Cassandra node. But if you want to load data from a Cassandra node, there is a simpler solution: you can use the JMX-StorageService-bulkload() call from said node. -Ursprüngliche Nachricht- Von: Rahul Neelakantan [mailto:ra...@rahul.be] Gesendet: Donnerstag, 31. Juli 2014 11:40 An: user@cassandra.apache.org Cc: cassandra-u...@incubator.apache.org Betreff: Re: Migration from Cassandra 1.2.5 to Cassandra 2.0.8 with changed partitioner settings You said you tried restoring a snapshot via bulk loader, did you actually run sstableloader? Rahul Neelakantan On Jul 31, 2014, at 2:54 AM, tsi thorsten.s...@t-systems.com wrote: Well, the new Cassandra cluster is already setup with the different partitioner settings and there are already other applications running on it. So the task is to migrate our application data to this new cluster to avoid setting up a dedicated Cassandra cluster just for our application. -- View this message in context: http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Migra tion-from-Cassandra-1-2-5-to-Cassandra-2-0-8-with-changed-partitioner- settings-tp7596019p7596062.html Sent from the cassandra-u...@incubator.apache.org mailing list archive at Nabble.com. -- *Ken Hancock *| System Architect, Advanced Advertising SeaChange International 50 Nagog Park Acton, Massachusetts 01720 ken.hanc...@schange.com | www.schange.com | NASDAQ:SEAC http://www.schange.com/en-US/Company/InvestorRelations.aspx Office: +1 (978) 889-3329 | [image: Google Talk:] ken.hanc...@schange.com | [image: Skype:]hancockks | [image: Yahoo IM:]hancockks [image: LinkedIn] http://www.linkedin.com/in/kenhancock [image: SeaChange International] http://www.schange.com/This e-mail and any attachments may contain information which is SeaChange International confidential. The information enclosed is intended only for the addressees herein and may not be copied or forwarded without permission from SeaChange International.
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
On Thu, Aug 7, 2014 at 12:27 PM, Robert Coli rc...@eventbrite.com wrote: On Thu, Aug 7, 2014 at 10:04 AM, srmore comom...@gmail.com wrote: Sorry for being ambiguous. By deletes I mean that running decommission I can no longer see any keyspaces owned by this node or replicated by other nodes using the cfstats command. I am also seeing the same behavior when I remove a single node from a cluster (without datacenters). I'm still not fully parsing you, but clusters should never forget schema as a result of decommission. Is that what you are saying is happening? Yes, this is what is happening. (In fact, even the decommissioned node itself does not forget its schema, which I personally consider a bug.) Ok, so I am assuming this is not a normal behavior and possibly a bug - is this correct ? =Rob
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
My reading is it didn't forget the schema. It lost the data. My reading is decomissioning worked fine. Possibly when you changed the replication on a keyspace to include a second data center, the data didn't get replicated. When you ADD a datacenter, you need to do a nodetool rebuild to get the data streamed to the new data center. When you alter a keyspace to include another datacenter in its replication schema, a nodetool repair is required -- was this done? http://www.datastax.com/documentation/cql/3.0/cql/cql_using/update_ks_rf_t.html When you use nodetool decomission, you're effectively deleting the parititioning token from the cluster. The node being decommissioned will stream its data to the new owners of its original token range. This streaming in no way should affect any other datacenter because you have not changed the tokens or data ownership for any datacenter but the one in which you are decomissioning a node. When you eventually decomission the last node in the datacenter, all data is gone as there are no tokens in that datacenter to own any data. If you had a keyspace that was only replicated within that datacenter, that data is gone (though you could probably add nodes back in and ressurect it). If you had a keyspace where you changed the replication to include another datacenter, if that datacenter had never received the data, then it may have the schema but would have none of the data (other than new data that was written AFTER you change the replication). On Thu, Aug 7, 2014 at 2:11 PM, srmore comom...@gmail.com wrote: On Thu, Aug 7, 2014 at 12:27 PM, Robert Coli rc...@eventbrite.com wrote: On Thu, Aug 7, 2014 at 10:04 AM, srmore comom...@gmail.com wrote: Sorry for being ambiguous. By deletes I mean that running decommission I can no longer see any keyspaces owned by this node or replicated by other nodes using the cfstats command. I am also seeing the same behavior when I remove a single node from a cluster (without datacenters). I'm still not fully parsing you, but clusters should never forget schema as a result of decommission. Is that what you are saying is happening? Yes, this is what is happening. (In fact, even the decommissioned node itself does not forget its schema, which I personally consider a bug.) Ok, so I am assuming this is not a normal behavior and possibly a bug - is this correct ? =Rob -- *Ken Hancock *| System Architect, Advanced Advertising SeaChange International 50 Nagog Park Acton, Massachusetts 01720 ken.hanc...@schange.com | www.schange.com | NASDAQ:SEAC http://www.schange.com/en-US/Company/InvestorRelations.aspx Office: +1 (978) 889-3329 | [image: Google Talk:] ken.hanc...@schange.com | [image: Skype:]hancockks | [image: Yahoo IM:]hancockks [image: LinkedIn] http://www.linkedin.com/in/kenhancock [image: SeaChange International] http://www.schange.com/This e-mail and any attachments may contain information which is SeaChange International confidential. The information enclosed is intended only for the addressees herein and may not be copied or forwarded without permission from SeaChange International.
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
Thanks for the detailed reply Ken, this really helps. I also realized that I wasn't doing a 'nodetool rebuild' after reading your email. I was following the steps mentioned here http://www.datastax.com/documentation/cassandra/1.2/cassandra/operations/ops_decomission_dc_t.html I do a test with nodetool rebuild and see what happens. On Thu, Aug 7, 2014 at 1:27 PM, Ken Hancock ken.hanc...@schange.com wrote: My reading is it didn't forget the schema. It lost the data. My reading is decomissioning worked fine. Possibly when you changed the replication on a keyspace to include a second data center, the data didn't get replicated. Probably not because I could see the sstables for the keyspace from the other datacenter created. My understanding could be wrong though. When you ADD a datacenter, you need to do a nodetool rebuild to get the data streamed to the new data center. When you alter a keyspace to include another datacenter in its replication schema, a nodetool repair is required -- was this done? http://www.datastax.com/documentation/cql/3.0/cql/cql_using/update_ks_rf_t.html I missed the 'nodetool rebuild' step that could be my issue, yes I did run repair. When you use nodetool decomission, you're effectively deleting the parititioning token from the cluster. The node being decommissioned will stream its data to the new owners of its original token range. This streaming in no way should affect any other datacenter because you have not changed the tokens or data ownership for any datacenter but the one in which you are decomissioning a node. That is what my understanding was, but when I decommission it does clear out (removes) all the keyspaces. When you eventually decomission the last node in the datacenter, all data is gone as there are no tokens in that datacenter to own any data. If you had a keyspace that was only replicated within that datacenter, that data is gone (though you could probably add nodes back in and ressurect it). The (now outdated) documentation [1] says that data remains on the node even after decommissioning. So I do not understand why the data would go away. If you had a keyspace where you changed the replication to include another datacenter, if that datacenter had never received the data, then it may have the schema but would have none of the data (other than new data that was written AFTER you change the replication). I would expect the repair to fix this, i.e. to stream the old data to the newly added datacenter. So, does nodetool rebuild help here ? [1] https://wiki.apache.org/cassandra/Operations#Removing_nodes_entirely On Thu, Aug 7, 2014 at 2:11 PM, srmore comom...@gmail.com wrote: On Thu, Aug 7, 2014 at 12:27 PM, Robert Coli rc...@eventbrite.com wrote: On Thu, Aug 7, 2014 at 10:04 AM, srmore comom...@gmail.com wrote: Sorry for being ambiguous. By deletes I mean that running decommission I can no longer see any keyspaces owned by this node or replicated by other nodes using the cfstats command. I am also seeing the same behavior when I remove a single node from a cluster (without datacenters). I'm still not fully parsing you, but clusters should never forget schema as a result of decommission. Is that what you are saying is happening? Yes, this is what is happening. (In fact, even the decommissioned node itself does not forget its schema, which I personally consider a bug.) Ok, so I am assuming this is not a normal behavior and possibly a bug - is this correct ? =Rob -- *Ken Hancock *| System Architect, Advanced Advertising SeaChange International 50 Nagog Park Acton, Massachusetts 01720 ken.hanc...@schange.com | www.schange.com | NASDAQ:SEAC http://www.schange.com/en-US/Company/InvestorRelations.aspx Office: +1 (978) 889-3329 | [image: Google Talk:] ken.hanc...@schange.com | [image: Skype:]hancockks | [image: Yahoo IM:]hancockks [image: LinkedIn] http://www.linkedin.com/in/kenhancock [image: SeaChange International] http://www.schange.com/This e-mail and any attachments may contain information which is SeaChange International confidential. The information enclosed is intended only for the addressees herein and may not be copied or forwarded without permission from SeaChange International.
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
I tried using 'nodetool rebuild' after I add the datacenters,date same outcome, and after I decommission my keyspaces are getting wiped out, I don't understand this. On Thu, Aug 7, 2014 at 1:54 PM, srmore comom...@gmail.com wrote: Thanks for the detailed reply Ken, this really helps. I also realized that I wasn't doing a 'nodetool rebuild' after reading your email. I was following the steps mentioned here http://www.datastax.com/documentation/cassandra/1.2/cassandra/operations/ops_decomission_dc_t.html I do a test with nodetool rebuild and see what happens. On Thu, Aug 7, 2014 at 1:27 PM, Ken Hancock ken.hanc...@schange.com wrote: My reading is it didn't forget the schema. It lost the data. My reading is decomissioning worked fine. Possibly when you changed the replication on a keyspace to include a second data center, the data didn't get replicated. Probably not because I could see the sstables for the keyspace from the other datacenter created. My understanding could be wrong though. When you ADD a datacenter, you need to do a nodetool rebuild to get the data streamed to the new data center. When you alter a keyspace to include another datacenter in its replication schema, a nodetool repair is required -- was this done? http://www.datastax.com/documentation/cql/3.0/cql/cql_using/update_ks_rf_t.html I missed the 'nodetool rebuild' step that could be my issue, yes I did run repair. When you use nodetool decomission, you're effectively deleting the parititioning token from the cluster. The node being decommissioned will stream its data to the new owners of its original token range. This streaming in no way should affect any other datacenter because you have not changed the tokens or data ownership for any datacenter but the one in which you are decomissioning a node. That is what my understanding was, but when I decommission it does clear out (removes) all the keyspaces. When you eventually decomission the last node in the datacenter, all data is gone as there are no tokens in that datacenter to own any data. If you had a keyspace that was only replicated within that datacenter, that data is gone (though you could probably add nodes back in and ressurect it). The (now outdated) documentation [1] says that data remains on the node even after decommissioning. So I do not understand why the data would go away. If you had a keyspace where you changed the replication to include another datacenter, if that datacenter had never received the data, then it may have the schema but would have none of the data (other than new data that was written AFTER you change the replication). I would expect the repair to fix this, i.e. to stream the old data to the newly added datacenter. So, does nodetool rebuild help here ? [1] https://wiki.apache.org/cassandra/Operations#Removing_nodes_entirely On Thu, Aug 7, 2014 at 2:11 PM, srmore comom...@gmail.com wrote: On Thu, Aug 7, 2014 at 12:27 PM, Robert Coli rc...@eventbrite.com wrote: On Thu, Aug 7, 2014 at 10:04 AM, srmore comom...@gmail.com wrote: Sorry for being ambiguous. By deletes I mean that running decommission I can no longer see any keyspaces owned by this node or replicated by other nodes using the cfstats command. I am also seeing the same behavior when I remove a single node from a cluster (without datacenters). I'm still not fully parsing you, but clusters should never forget schema as a result of decommission. Is that what you are saying is happening? Yes, this is what is happening. (In fact, even the decommissioned node itself does not forget its schema, which I personally consider a bug.) Ok, so I am assuming this is not a normal behavior and possibly a bug - is this correct ? =Rob -- *Ken Hancock *| System Architect, Advanced Advertising SeaChange International 50 Nagog Park Acton, Massachusetts 01720 ken.hanc...@schange.com | www.schange.com | NASDAQ:SEAC http://www.schange.com/en-US/Company/InvestorRelations.aspx Office: +1 (978) 889-3329 | [image: Google Talk:] ken.hanc...@schange.com | [image: Skype:]hancockks | [image: Yahoo IM:]hancockks [image: LinkedIn] http://www.linkedin.com/in/kenhancock [image: SeaChange International] http://www.schange.com/This e-mail and any attachments may contain information which is SeaChange International confidential. The information enclosed is intended only for the addressees herein and may not be copied or forwarded without permission from SeaChange International.
Re: Decommissioning a datacenter deletes the data (on decommissioned datacenter)
On Thu, Aug 7, 2014 at 12:29 PM, srmore comom...@gmail.com wrote: I tried using 'nodetool rebuild' after I add the datacenters,date same outcome, and after I decommission my keyspaces are getting wiped out, I don't understand this. File a JIRA with steps to reproduce the issue, and someone will tell you if there is any valid reason for the behavior you are seeing. http://issues.apache.org =Rob
Is nodetool cleanup necessary on nodes of different data center when new node is added
I plan to have a multi data center Cassandra 2 setup with 2-4 nodes per data center and several 10s of data centers. We have keyspaces replicated on a certain number of nodes on *each* data center. Essentially, each data center has a logical ring that covers all token ranges. We have a vnode based deployment. So tokens should get assigned to the nodes automatically. Documentation at http://www.datastax.com/documentation/cassandra/2.0/cassandra/operations/ops_add_node_to_cluster_t.html suggests that addition of new node requires cleanup to be run on all other nodes of the cluster. However, it does not clarify the procedure in a multi-data center setup. My understanding is that nodetool cleanup removes data which no longer belongs to that node. When a new data center is being setup, we are creating completely new replicas and AFAICT, it does not result in data movement/rebalance outside of this new data center and hence there is no cleanup requirement on nodes of other data centers. Is someone able to confirm if my understanding is right, and cleanup is not required on nodes of other data centers? Thanks Vish
Re: Is nodetool cleanup necessary on nodes of different data center when new node is added
On Thu, Aug 7, 2014 at 2:46 PM, Viswanathan Ramachandran vish.ramachand...@gmail.com wrote: I plan to have a multi data center Cassandra 2 setup with 2-4 nodes per data center and several 10s of data centers. We have My understanding is that nodetool cleanup removes data which no longer belongs to that node. When a new data center is being setup, we are creating completely new replicas and AFAICT, it does not result in data movement/rebalance outside of this new data center and hence there is no cleanup requirement on nodes of other data centers. Is someone able to confirm if my understanding is right, and cleanup is not required on nodes of other data centers? This is the correct understanding; as you say, the key is that cleanup removes data which no longer belongs to a node. In this situation, no node loses responsibility for any replica, so cleanup is not necessary. For what it's worth, 2-4 nodes per data center and several 10s of data centers is an unusual deploy for Cassandra. If you can discuss it, what is the use case? =Rob