Broken keyspace strategy_option with zero replicas --------------------------------------------------
Key: CASSANDRA-1924 URL: https://issues.apache.org/jira/browse/CASSANDRA-1924 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7.0 rc 3, 0.7.0 rc 2, 0.7.0 rc 1, 0.7 beta 3, 0.7 beta 2, 0.7 beta 1 Reporter: Thor Carpenter Priority: Minor When a keyspace is defined that has strategy options specifying zero replicas should be place in a datacenter (e.g. DCX:0), an assert is violated for any insert and LOCAL_QUORUM reads fail. I'm not sure if the issue is that there are no nodes in DCX or that I'm saying DCX shouldn't get any replicas, or a combination of the two. The broken keyspace: create keyspace KeyspaceDC1 with replication_factor = 1 and placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' and strategy_options = [{DC1:1, DC2:0}]; The fixed keyspace: create keyspace KeyspaceDC1 with replication_factor = 1 and placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' and strategy_options = [{DC1:1}]; To reproduce: * Install the 0.7rc3 rpm on a single node in "DC1". * In cassandra.yaml set initial_token = 1 and specify PropertyFileSnitch. * cassandra-topology.properties: 10.5.64.26=DC1:R1 default=DC2:R1 * Schema loaded via cassandra-cli: create keyspace KeyspaceDC1 with replication_factor = 1 and placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' and strategy_options = [{DC1:1, DC2:0}]; use KeyspaceDC1; create column family TestCF with column_type = 'Standard' and comparator = 'BytesType' and keys_cached = 200000 and rows_cached = 2000 and gc_grace = 0 and read_repair_chance = 0.0; * In cassandra-cli execute the following: [defa...@unknown] use KeyspaceDC1; Authenticated to keyspace: KeyspaceDC1 [defa...@keyspacedc1] set TestCF['some key']['some col'] = 'some value'; Internal error processing insert * If you have asserts enabled, check system.log where you should find the assertion error: DEBUG [pool-1-thread-3] 2010-12-29 12:10:38,897 CassandraServer.java (line 362) insert ERROR [pool-1-thread-3] 2010-12-29 12:10:38,906 Cassandra.java (line 2960) Internal error processing insert java.lang.AssertionError at org.apache.cassandra.locator.TokenMetadata.firstTokenIndex(TokenMetadata.java:392) at org.apache.cassandra.locator.TokenMetadata.ringIterator(TokenMetadata.java:417) at org.apache.cassandra.locator.NetworkTopologyStrategy.calculateNaturalEndpoints(NetworkTopologyStrategy.java:95) at org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:99) at org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1411) at org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1394) at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:109) at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:442) at org.apache.cassandra.thrift.CassandraServer.insert(CassandraServer.java:379) at org.apache.cassandra.thrift.Cassandra$Processor$insert.process(Cassandra.java:2952) at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:167) 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:619) * If you don't have asserts enabled, you should find that no errors are logged but LOCAL_QUORUM reads cause TimedOutExceptions on the client. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.