[ 
https://issues.apache.org/jira/browse/CASSANDRA-1831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jonathan Ellis resolved CASSANDRA-1831.
---------------------------------------

    Resolution: Duplicate

The right fix is to not allow conflicting definitions in the first place.  
CASSANDRA-1263 is open for this.

> NetworkTopologyStrategy allows mismatched RF resulting in obscure failures
> --------------------------------------------------------------------------
>
>                 Key: CASSANDRA-1831
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1831
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7.0 rc 1
>            Reporter: Peter Schuller
>
> On today's 0.7 branch:
> Creating a keyspace like this (not how to do it in production, but that's not 
> the point):
>    create keyspace MyKeySpace with replication_factor = 2 and 
> placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy';
> This is accepted by Cassandra in spite of there being no strategy options. 
> Describing the keyspace will then give output similar to:
> Keyspace: MyKeySpace:
>  Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
> null
> Attempts to write and read respectively gives the errors included at the 
> bottom of this comment.
> What happens is that the NTS's getReplicationFactor() returns the sum of RF 
> for each DC. But lacking any replicate placement options for DC:s, the sum 
> will always be 0. The result is that NTS.calculateNaturalEndpoints() yields 0 
> endpoints thus triggering the assertion failures apparent in the strack 
> traces.
> This was caused by misconfiguration during testing but should be handled 
> better. What are people's thoughts on the set of changes that would 
> constitute a proper fix?
> Is there a reason for NTS to ever conclude that RF is different than that of 
> the CF def? If not, I would say that one fix is to make the NTS bail early if 
> the calculated RF adding up the DC placements does not match the configured 
> RF for the column family. (I'll submit a patch if people agree.)
> Beyond that, what else, if anything should be done? Should the creation fail 
> due to the RF being inconsistent with strategy options? Is it correct that 
> code assumes that naturalEndPoints will never return fewer nodes than RF? It 
> seems natural to me that the natural endpoint count should always match RF, 
> unless the total number of nodes in the cluster is lacking. But this gets 
> complicated with NTS since the requirement is suddenly that you have enough 
> in each DC. This probably relates to previous discussions on whether or not 
> to allow an RF which is higher than the number of nodes in a cluster.
> In this case, we failed hard because we got exactly 0 endpoints and triggered 
> assertions. In other cases we might have gotten say 1, in which case we may 
> have successfully been able to read and write as if we had a lower RF even 
> though the column family RF was set to 2. This seems dangerous.
> ERROR [pool-1-thread-2] 2010-12-07 11:18:40,638 Cassandra.java (line
> 3044) Internal error processing batch_mutate
> java.lang.AssertionError: invalid response count 1 for replication factor 0
>        at 
> org.apache.cassandra.service.WriteResponseHandler.determineBlockFor(WriteResponseHandler.java:98)
>        at 
> org.apache.cassandra.service.WriteResponseHandler.<init>(WriteResponseHandler.java:48)
>        at 
> org.apache.cassandra.service.WriteResponseHandler.create(WriteResponseHandler.java:61)
>        at 
> org.apache.cassandra.locator.AbstractReplicationStrategy.getWriteResponseHandler(AbstractReplicationStrategy.java:125)
>        at 
> org.apache.cassandra.locator.NetworkTopologyStrategy.getWriteResponseHandler(NetworkTopologyStrategy.java:166)
>        at 
> org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:114)
>        at 
> org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:446)
>        at 
> org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:419)
>        at 
> org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.process(Cassandra.java:3036)
>        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:662)
> ERROR [pool-1-thread-3] 2010-12-07 11:18:50,474 Cassandra.java (line
> 2876) Internal error processing get_range_slices
> java.lang.AssertionError
>        at 
> org.apache.cassandra.service.RangeSliceResponseResolver.<init>(RangeSliceResponseResolver.java:53)
>        at 
> org.apache.cassandra.service.StorageProxy.getRangeSlice(StorageProxy.java:450)
>        at 
> org.apache.cassandra.thrift.CassandraServer.get_range_slices(CassandraServer.java:507)
>        at 
> org.apache.cassandra.thrift.Cassandra$Processor$get_range_slices.process(Cassandra.java:2868)
>        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:662)
>  INFO [MigrationStage:1] 2010-12-07 11:24:09,220

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to