You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Yiming Sun (JIRA)" <ji...@apache.org> on 2012/06/04 22:58:23 UTC

[jira] [Comment Edited] (CASSANDRA-4281) schema agreement accross the nodes

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

Yiming Sun edited comment on CASSANDRA-4281 at 6/4/12 8:57 PM:
---------------------------------------------------------------

Actually this bug is different than CASSANDRA-4269 and has not been fixed in v1.1.1.  What makes us (me included) think it was the same as CASSANDRA-4269 is the exception thrown in version 1.1.0.  But in v1.1.1, it throws a different exception, so I would suggest to reopen this issue ticket.

I just downloaded and installed cassandra 1.1.1 to make sure the issue was fixed, but it wasn't.  We have a 6-node Cassandra cluster and I loaded the schema (listed below) to one of the nodes via cassandra-cli.  The print-out message says schema agree across the cluster for all column families, but when I use cassandra-cli to "show keyspaces" on the nodes one by one, I found at least one node was missing some column families (the listing below shows I did it twice and both times it failed on the same node, and none of the column families were properly defined, but sometimes some column families would show up but others would be missing):

...
Keyspace: TestKSpace:
  Replication Strategy: org.apache.cassandra.locator.NetworkTopologyStrategy
  Durable Writes: true
    Options: [DC1:2]
  Column Families:
Keyspace: TestKeySpace:
  Replication Strategy: org.apache.cassandra.locator.NetworkTopologyStrategy
  Durable Writes: true
    Options: [DC1:2]
  Column Families:
...

the system.log has the following exception, which is different than the exception we previously thought from CASSANDRA-4269


ERROR [MigrationStage:1] 2012-06-04 16:47:30,802 AbstractCassandraDaemon.java (line 134) Exception in thread Thread[MigrationStage:1,5,main]
org.apache.cassandra.db.marshal.MarshalException: invalid UTF8 bytes 4fcd1ef6
        at org.apache.cassandra.db.marshal.UTF8Type.getString(UTF8Type.java:56)
        at org.apache.cassandra.cql3.ColumnIdentifier.<init>(ColumnIdentifier.java:47)
        at org.apache.cassandra.cql3.CFDefinition.getKeyId(CFDefinition.java:125)
        at org.apache.cassandra.cql3.CFDefinition.<init>(CFDefinition.java:59)
        at org.apache.cassandra.config.CFMetaData.updateCfDef(CFMetaData.java:1303)
        at org.apache.cassandra.config.CFMetaData.keyAlias(CFMetaData.java:224)
        at org.apache.cassandra.config.CFMetaData.fromSchemaNoColumns(CFMetaData.java:1187)
        at org.apache.cassandra.config.CFMetaData.fromSchema(CFMetaData.java:1215)
        at org.apache.cassandra.config.KSMetaData.deserializeColumnFamilies(KSMetaData.java:291)
        at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:396)
        at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:271)
        at org.apache.cassandra.db.DefsTable.mergeRemoteSchema(DefsTable.java:249)
        at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:48)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        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)


And here is the schema:

create keyspace TestKeySpace
    with strategy_options={DC1:2}
    and placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy';

use TestKeySpace;

create column family VolumeContents
    with column_type = Standard
    and key_validation_class = UTF8Type
    and comparator = UTF8Type;

create column family Collections
    with column_type = Standard
    and key_validation_class = UTF8Type
    and comparator = UTF8Type;

create column family CollectionNames
    with column_type = Standard
    and key_validation_class = UTF8Type
    and comparator = UTF8Type;


                
      was (Author: yimingsun):
    Actually this bug is different than CASSANDRA-4269 and has not been fixed in v1.1.1.  What makes us (me included) think it was the same as CASSANDRA-4269 is the exception thrown.

I just downloaded and installed cassandra 1.1.1 to make sure the issue was fixed, but it wasn't.  We have a 6-node Cassandra cluster and I loaded the schema (listed below) to one of the nodes via cassandra-cli.  The print-out message says schema agree across the cluster for all column families, but when I use cassandra-cli to "show keyspaces" on the nodes one by one, I found at least one node was missing some column families (the listing below shows I did it twice and both times it failed on the same node, and none of the column families were properly defined, but sometimes some column families would show up but others would be missing):

...
Keyspace: TestKSpace:
  Replication Strategy: org.apache.cassandra.locator.NetworkTopologyStrategy
  Durable Writes: true
    Options: [DC1:2]
  Column Families:
Keyspace: TestKeySpace:
  Replication Strategy: org.apache.cassandra.locator.NetworkTopologyStrategy
  Durable Writes: true
    Options: [DC1:2]
  Column Families:
...

the system.log has the following exception, which is different than the exception we previously thought from CASSANDRA-4269


ERROR [MigrationStage:1] 2012-06-04 16:47:30,802 AbstractCassandraDaemon.java (line 134) Exception in thread Thread[MigrationStage:1,5,main]
org.apache.cassandra.db.marshal.MarshalException: invalid UTF8 bytes 4fcd1ef6
        at org.apache.cassandra.db.marshal.UTF8Type.getString(UTF8Type.java:56)
        at org.apache.cassandra.cql3.ColumnIdentifier.<init>(ColumnIdentifier.java:47)
        at org.apache.cassandra.cql3.CFDefinition.getKeyId(CFDefinition.java:125)
        at org.apache.cassandra.cql3.CFDefinition.<init>(CFDefinition.java:59)
        at org.apache.cassandra.config.CFMetaData.updateCfDef(CFMetaData.java:1303)
        at org.apache.cassandra.config.CFMetaData.keyAlias(CFMetaData.java:224)
        at org.apache.cassandra.config.CFMetaData.fromSchemaNoColumns(CFMetaData.java:1187)
        at org.apache.cassandra.config.CFMetaData.fromSchema(CFMetaData.java:1215)
        at org.apache.cassandra.config.KSMetaData.deserializeColumnFamilies(KSMetaData.java:291)
        at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:396)
        at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:271)
        at org.apache.cassandra.db.DefsTable.mergeRemoteSchema(DefsTable.java:249)
        at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:48)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        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)


And here is the schema:

create keyspace TestKeySpace
    with strategy_options={DC1:2}
    and placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy';

use TestKeySpace;

create column family VolumeContents
    with column_type = Standard
    and key_validation_class = UTF8Type
    and comparator = UTF8Type;

create column family Collections
    with column_type = Standard
    and key_validation_class = UTF8Type
    and comparator = UTF8Type;

create column family CollectionNames
    with column_type = Standard
    and key_validation_class = UTF8Type
    and comparator = UTF8Type;


                  
> schema agreement accross the nodes
> ----------------------------------
>
>                 Key: CASSANDRA-4281
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4281
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.1.0
>            Reporter: Claudio Atzori
>
> I'm creating a cluster of 2 nodes (for now), of cassandra 1.1.0, installed on Ubuntu 10.04
> root@node2.d:/etc/cassandra# uname -a
> Linux node2 2.6.32-5-xen-amd64 #1 SMP Fri Sep 9 22:23:19 UTC 2011 x86_64 GNU/Linux
> with all defaults in cassandra.yaml, except for:
> cluster_name
> initial_token (I set a 50/50 balancing between the 2 nodes)
> seeds list (one of the 2 nodes ip address)
> #listen_address: localhost
> #rpc_address: localhost
> The 2 nodes recognize each other
> root@node2.d:/etc/cassandra# nodetool ring
> Address         DC          Rack        Status State   Load            Effective-Owership  Token                                       
>                                                                                            85070591730234615865843651857942052864      
> 146.48.122.136  datacenter1 rack1       Up     Normal  28.62 KB        100.00%             0}}                                           
> 146.48.122.137  datacenter1 rack1       Up     Normal  21.79 KB        100.00%             85070591730234615865843651857942052864
> But, I'm experiencing an issue. I'm trying to define a new keyspace from the cqlsh.
> cqlsh> CREATE KEYSPACE efg_mr WITH strategy_class = 'SimpleStrategy' AND strategy_options:replication_factor=2 ;
> ..and ok, the new keyspace is seen accross the 2 nodes. 
> cqlsh> DESCRIBE KEYSPACE efg_mr ;
> CREATE KEYSPACE efg_mr WITH strategy_class = 'SimpleStrategy'
>   AND strategy_options:replication_factor = '2';
> now I wanted to define a column family:
> cqlsh> CREATE COLUMNFAMILY records (KEY varchar PRIMARY KEY, title varchar, year varchar) ;
> at this point I noticed an exception in /var/log/cassandra/output.log
> ERROR 14:28:47,475 Exception in thread Thread[MigrationStage:1,5,main]
> java.lang.RuntimeException: java.nio.charset.MalformedInputException: Input length = 1
> 	at org.apache.cassandra.cql3.ColumnIdentifier.<init>(ColumnIdentifier.java:50)
> 	at org.apache.cassandra.cql3.CFDefinition.getKeyId(CFDefinition.java:125)
> 	at org.apache.cassandra.cql3.CFDefinition.<init>(CFDefinition.java:59)
> 	at org.apache.cassandra.config.CFMetaData.updateCfDef(CFMetaData.java:1278)
> 	at org.apache.cassandra.config.CFMetaData.keyAlias(CFMetaData.java:221)
> 	at org.apache.cassandra.config.CFMetaData.fromSchemaNoColumns(CFMetaData.java:1162)
> 	at org.apache.cassandra.config.CFMetaData.fromSchema(CFMetaData.java:1190)
> 	at org.apache.cassandra.config.KSMetaData.deserializeColumnFamilies(KSMetaData.java:291)
> 	at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:358)
> 	at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:270)
> 	at org.apache.cassandra.db.DefsTable.mergeRemoteSchema(DefsTable.java:248)
> 	at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:48)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	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)
> Caused by: java.nio.charset.MalformedInputException: Input length = 1
> 	at java.nio.charset.CoderResult.throwException(CoderResult.java:260)
> 	at java.nio.charset.CharsetDecoder.decode(CharsetDecoder.java:781)
> 	at org.apache.cassandra.utils.ByteBufferUtil.string(ByteBufferUtil.java:163)
> 	at org.apache.cassandra.utils.ByteBufferUtil.string(ByteBufferUtil.java:120)
> 	at org.apache.cassandra.cql3.ColumnIdentifier.<init>(ColumnIdentifier.java:46)
> 	... 18 more
> and from now on, only one of the 2 nodes knows about the new column family, the other one somehow hasn't been informed, or didn't complete the agreement on the new column family.
> Since I'm creating a new cluster I tried several times to drop all the data (rm -rf /var/lib/cassandra/*) and starting over again. But sometimes this error happens on the column family definition, sometimes after a CREATE INDEX command.
> Am I doing something wrong?

--
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