You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Casey Deccio <ca...@deccio.net> on 2012/05/15 21:08:26 UTC

cassandra upgrade to 1.1 - migration problem

I recently upgraded from cassandra 1.0.10 to 1.1.  Everything worked fine
in one environment, but after I upgraded in another, I can't find my
keyspace.  When I run, e.g., cassandra-cli with 'use KeySpace;' It tells me
that the keyspace doesn't exist.  In the log I see this:

ERROR [MigrationStage:1] 2012-05-15 11:39:48,216
AbstractCassandraDaemon.java (line 134) Exception in thread
Thread[MigrationStage:1,5,main]java.lang.AssertionError
        at
org.apache.cassandra.db.DefsTable.updateKeyspace(DefsTable.java:441)
        at
org.apache.cassandra.db.DefsTable.mergeKeyspaces(DefsTable.java:339)
        at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:269)
        at
org.apache.cassandra.db.DefsTable.mergeRemoteSchema(DefsTable.java:248)
        at
org.apache.cassandra.service.MigrationManager$MigrationTask.runMayThrow(MigrationManager.java:416)
        at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:636)

I can see that the data I would expect still seems to be in the new place
(/var/lib/cassandra/data/App/ColFamily/App-DomainName-*) on all nodes.

What am I missing?

Thanks,
Casey

Re: cassandra upgrade to 1.1 - migration problem

Posted by Casey Deccio <ca...@deccio.net>.
On Tue, May 15, 2012 at 5:41 PM, Dave Brosius <db...@mebigfatguy.com>wrote:

> The replication factor for a keyspace is stored in the
> system.schema_keyspaces column family.
>
> Since you can't view this with cli as the server won't start, the only way
> to look at it, that i know of is to use the
>
> sstable2json tool on the *.db file for that column family...
>
> So for instance on my machine i do
>
> ./sstable2json /var/lib/cassandra/data/**system/schema_keyspaces/**
> system-schema_keyspaces-ia-1-**Data.db
>
> and get
>
>
> {
> "7374726573735f6b73": [["durable_writes","true",**1968197311980145],
> ["name","stress_ks",**1968197311980145], ["strategy_class","org.apache.**
> cassandra.locator.**SimpleStrategy",**1968197311980145],
> ["strategy_options","{\"**replication_factor\":\"3\"}",**
> 1968197311980145]]
>
> It's likely you don't have a entry from replication_factor.
>
>
Yep, I checked the system.schema_keyspaces ColumnFamily, and there was no
replication_factor value, as you suspected.  But the dev cluster that
worked after upgrade did have that value, so it started up okay.
Apparently pre-1.1 was less picky about its presence.


> Theoretically i suppose you could embellish the output, and use
> json2sstable to fix it, but I have no experience here, and would get the
> blessings of datastax fellas, before proceeding.


Actually, I went ahead and took a chance because I had already completely
offline for several hours and wanted to get things back up.  I did what you
suggested and added the replication_factor value to the json returned from
sstable2json and imported it using json2sstable.  Fortunately I had the dev
cluster values to use as a basis.  I started things up, and it worked like
a champ.  Thanks!

Casey

Re: cassandra upgrade to 1.1 - migration problem

Posted by Dave Brosius <db...@mebigfatguy.com>.
The replication factor for a keyspace is stored in the 
system.schema_keyspaces column family.

Since you can't view this with cli as the server won't start, the only 
way to look at it, that i know of is to use the

sstable2json tool on the *.db file for that column family...

So for instance on my machine i do

./sstable2json 
/var/lib/cassandra/data/system/schema_keyspaces/system-schema_keyspaces-ia-1-Data.db

and get


{
"7374726573735f6b73": [["durable_writes","true",1968197311980145], 
["name","stress_ks",1968197311980145], 
["strategy_class","org.apache.cassandra.locator.SimpleStrategy",1968197311980145], 
["strategy_options","{\"replication_factor\":\"3\"}",1968197311980145]]

It's likely you don't have a entry from replication_factor.

Theoretically i suppose you could embellish the output, and use 
json2sstable to fix it, but I have no experience here, and would get the 
blessings of datastax fellas, before proceeding.





On 05/15/2012 07:02 PM, Casey Deccio wrote:
> Sorry to reply to my own message (again).  I took a closer look at the 
> logs and realized that the partitioner errors aren't what kept the 
> daemon to stop; those errors are in the logs even before I upgraded.  
> This one seems to be the culprit.
>
> java.lang.reflect.InvocationTargetException
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at 
> org.apache.commons.daemon.support.DaemonLoader.load(DaemonLoader.java:160)
> Caused by: 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.init(AbstractCassandraDaemon.java:254)
>         ... 5 more
> 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)
>         ... 9 more
> Cannot load daemon
>
> I'm not sure how to check the replication_factor and/or update it 
> without using cassandra-cli, which requires the daemon to be running.
>
> Casey


Re: cassandra upgrade to 1.1 - migration problem

Posted by Casey Deccio <ca...@deccio.net>.
Sorry to reply to my own message (again).  I took a closer look at the logs
and realized that the partitioner errors aren't what kept the daemon to
stop; those errors are in the logs even before I upgraded.  This one seems
to be the culprit.

java.lang.reflect.InvocationTargetException
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
        at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at
org.apache.commons.daemon.support.DaemonLoader.load(DaemonLoader.java:160)
Caused by: 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.init(AbstractCassandraDaemon.java:254)
        ... 5 more
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)
        ... 9 more
Cannot load daemon

I'm not sure how to check the replication_factor and/or update it without
using cassandra-cli, which requires the daemon to be running.

Casey

Re: cassandra upgrade to 1.1 - migration problem

Posted by Casey Deccio <ca...@deccio.net>.
cassandra.yaml on all nodes had ByteOrderedPartitioner with both the
previous version and upgraded version.

That being said, when I first started up cassandra after upgrading  (with
the updated .yaml, including ByteOrderedPartitioner) all nodes in the ring
appeared to be up.  But the load they carried was minimal (KB, as opposed
to GB in the previous version), and the keyspace didn't exist.  Then when I
attempted to restart the daemon on each to see if it would help, but
starting up failed on each with the partition error.

Casey

On Tue, May 15, 2012 at 12:59 PM, Oleg Dulin <oleg.dulin@liquidanalytics.com
> wrote:

> Did you check cassandra.yaml to make sure partitioner there matches what
> was in your old cluster ?
>
> Regards,
> Oleg Dulin
> Please note my new office #: 732-917-0159
>
> On May 15, 2012, at 3:22 PM, Casey Deccio wrote:
>
> Here's something new in the logs:
>
> ERROR 12:21:09,418 Exception in thread Thread[SSTableBatchOpen:2,5,main]
> java.lang.RuntimeException: Cannot open
> /var/lib/cassandra/data/system/Versions/system-Versions-hc-35 because
> partitioner does not match org.apache.cassandra.dht.ByteOrderedPartitioner
>         at
> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:164)
>         at
> org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:224)
>         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)
>
> Casey
>
> On Tue, May 15, 2012 at 12:08 PM, Casey Deccio <ca...@deccio.net> wrote:
>
>> I recently upgraded from cassandra 1.0.10 to 1.1.  Everything worked fine
>> in one environment, but after I upgraded in another, I can't find my
>> keyspace.  When I run, e.g., cassandra-cli with 'use KeySpace;' It tells me
>> that the keyspace doesn't exist.  In the log I see this:
>>
>> ERROR [MigrationStage:1] 2012-05-15 11:39:48,216
>> AbstractCassandraDaemon.java (line 134) Exception in thread
>> Thread[MigrationStage:1,5,main]java.lang.AssertionError
>>         at
>> org.apache.cassandra.db.DefsTable.updateKeyspace(DefsTable.java:441)
>>         at
>> org.apache.cassandra.db.DefsTable.mergeKeyspaces(DefsTable.java:339)
>>         at
>> org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:269)
>>         at
>> org.apache.cassandra.db.DefsTable.mergeRemoteSchema(DefsTable.java:248)
>>         at
>> org.apache.cassandra.service.MigrationManager$MigrationTask.runMayThrow(MigrationManager.java:416)
>>         at
>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>>         at
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>>         at
>> java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
>>         at
>> java.util.concurrent.FutureTask.run(FutureTask.java:166)        at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>> at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>> at java.lang.Thread.run(Thread.java:636)
>>
>> I can see that the data I would expect still seems to be in the new place
>> (/var/lib/cassandra/data/App/ColFamily/App-DomainName-*) on all nodes.
>>
>> What am I missing?
>>
>> Thanks,
>> Casey
>>
>
>
>

Re: cassandra upgrade to 1.1 - migration problem

Posted by Oleg Dulin <ol...@liquidanalytics.com>.
Did you check cassandra.yaml to make sure partitioner there matches what was in your old cluster ?

Regards,
Oleg Dulin
Please note my new office #: 732-917-0159

On May 15, 2012, at 3:22 PM, Casey Deccio wrote:

> Here's something new in the logs:
> 
> ERROR 12:21:09,418 Exception in thread Thread[SSTableBatchOpen:2,5,main]
> java.lang.RuntimeException: Cannot open /var/lib/cassandra/data/system/Versions/system-Versions-hc-35 because partitioner does not match org.apache.cassandra.dht.ByteOrderedPartitioner
>         at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:164)
>         at org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:224)
>         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)
> 
> Casey
> 
> On Tue, May 15, 2012 at 12:08 PM, Casey Deccio <ca...@deccio.net> wrote:
> I recently upgraded from cassandra 1.0.10 to 1.1.  Everything worked fine in one environment, but after I upgraded in another, I can't find my keyspace.  When I run, e.g., cassandra-cli with 'use KeySpace;' It tells me that the keyspace doesn't exist.  In the log I see this:
> 
> ERROR [MigrationStage:1] 2012-05-15 11:39:48,216 AbstractCassandraDaemon.java (line 134) Exception in thread Thread[MigrationStage:1,5,main]java.lang.AssertionError
>         at org.apache.cassandra.db.DefsTable.updateKeyspace(DefsTable.java:441)
>         at org.apache.cassandra.db.DefsTable.mergeKeyspaces(DefsTable.java:339)
>         at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:269)
>         at org.apache.cassandra.db.DefsTable.mergeRemoteSchema(DefsTable.java:248)
>         at org.apache.cassandra.service.MigrationManager$MigrationTask.runMayThrow(MigrationManager.java:416)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:166)        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)        at java.lang.Thread.run(Thread.java:636)
> 
> I can see that the data I would expect still seems to be in the new place (/var/lib/cassandra/data/App/ColFamily/App-DomainName-*) on all nodes.
> 
> What am I missing?
> 
> Thanks,
> Casey
> 


Re: cassandra upgrade to 1.1 - migration problem

Posted by Casey Deccio <ca...@deccio.net>.
Here's something new in the logs:

ERROR 12:21:09,418 Exception in thread Thread[SSTableBatchOpen:2,5,main]
java.lang.RuntimeException: Cannot open
/var/lib/cassandra/data/system/Versions/system-Versions-hc-35 because
partitioner does not match org.apache.cassandra.dht.ByteOrderedPartitioner
        at
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:164)
        at
org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:224)
        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)

Casey

On Tue, May 15, 2012 at 12:08 PM, Casey Deccio <ca...@deccio.net> wrote:

> I recently upgraded from cassandra 1.0.10 to 1.1.  Everything worked fine
> in one environment, but after I upgraded in another, I can't find my
> keyspace.  When I run, e.g., cassandra-cli with 'use KeySpace;' It tells me
> that the keyspace doesn't exist.  In the log I see this:
>
> ERROR [MigrationStage:1] 2012-05-15 11:39:48,216
> AbstractCassandraDaemon.java (line 134) Exception in thread
> Thread[MigrationStage:1,5,main]java.lang.AssertionError
>         at
> org.apache.cassandra.db.DefsTable.updateKeyspace(DefsTable.java:441)
>         at
> org.apache.cassandra.db.DefsTable.mergeKeyspaces(DefsTable.java:339)
>         at
> org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:269)
>         at
> org.apache.cassandra.db.DefsTable.mergeRemoteSchema(DefsTable.java:248)
>         at
> org.apache.cassandra.service.MigrationManager$MigrationTask.runMayThrow(MigrationManager.java:416)
>         at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>         at
> java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
>
> I can see that the data I would expect still seems to be in the new place
> (/var/lib/cassandra/data/App/ColFamily/App-DomainName-*) on all nodes.
>
> What am I missing?
>
> Thanks,
> Casey
>