[ 
https://issues.apache.org/jira/browse/CASSANDRA-4294?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13287268#comment-13287268
 ] 

Sylvain Lebresne commented on CASSANDRA-4294:
---------------------------------------------

Well, apparently we did broke compatibility of 0.7 -> 1.1. In 1.0, in 
{{KSMetadata.fromAvro()}}, we have the following line:
{noformat}
maybeAddReplicationFactor(strategyOptions, ks.strategy_class.toString(), 
ks.replication_factor);
{noformat}
which allows compatibility with the old ways where the replication factor was 
part of the KsDef directly. In 1.1, we've removed that line (the fromAvro 
method has been moved to config/Avro.java).

I think the idea of removing this was that we were fine not supporting direct 
upgrade from 0.7 -> 1.1 and force upgrade to 1.0 first. Unfortunately, since 
maybeAddReplicationFactor() was called by fromAvro, it means that one would 
have to upgrade from 0.7 to 1.0 *and* do some schema upgrade to every keyspace 
before upgrading to 1.1. Otherwise, I don't think that the fix of 
maybeAddReplicationFactor is persisted (I haven't tested so I'm not 100% sure 
but I think that's the case).

So I think we may want to add back the maybeAddReplicationFactor to fromAvro 
for now. We can drop that in 1.2 if we want however, since 1.1 will guarantee 
that this is persisted since the first thing 1.1 does is to save schema in the 
new format.
                
> Upgrading encounters: 'SimpleStrategy requires a replication_factor strategy 
> option.' and refuses to start
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-4294
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4294
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.1.0
>            Reporter: Brandon Williams
>             Fix For: 1.1.1
>
>
> I've seen this reported quite a few times now:
> {noformat}
> ERROR [main] 2012-05-29 19:33:40,589 AbstractCassandraDaemon.java (line 370) 
> Exception encountered during startup
> java.lang.RuntimeException: 
> org.apache.cassandra.config.ConfigurationException: SimpleStrategy requires a 
> replication_factor strategy option.
>   at org.apache.cassandra.db.Table.<init>(Table.java:275)
>   at org.apache.cassandra.db.Table.open(Table.java:114)
>   at org.apache.cassandra.db.Table.open(Table.java:97)
>   at 
> org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:204)
>   at 
> org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:353)
>   at 
> org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:106)
> Caused by: org.apache.cassandra.config.ConfigurationException: SimpleStrategy 
> requires a replication_factor strategy option.
>   at 
> org.apache.cassandra.locator.SimpleStrategy.validateOptions(SimpleStrategy.java:71)
>   at 
> org.apache.cassandra.locator.AbstractReplicationStrategy.createReplicationStrategy(AbstractReplicationStrategy.java:218)
>   at org.apache.cassandra.db.Table.createReplicationStrategy(Table.java:295)
>   at org.apache.cassandra.db.Table.<init>(Table.java:271)
>   ... 5 more
> {noformat}
> The common thread seems to be old lineage, from at least 0.7.  1.0.x works 
> fine, but upgrading to 1.1 causes the problem.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to