Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-11803

Creating a materialized view on a table with "token" column breaks the cluster

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Urgent
    • Resolution: Fixed
    • 3.0.10, 3.10, 4.0-alpha1, 4.0
    • None
    • Availability - Cluster Crash
    • Critical
    • 3.3

    Description

      On a new Cassandra cluster, if we create a table with a field called "token" (with quotes) and then create a materialized view that uses "token", the cluster breaks. A ServerError is returned, and no further nodetool operations on the sstables work. Restarting the Cassandra server will also fail. It seems like the entire cluster is hosed.

      We tried this on Cassandra 3.3 and 3.5.

      Here's how to produce (on an new, empty cassandra 3.5 docker container):

      [cqlsh 5.0.1 | Cassandra 3.5 | CQL spec 3.4.0 | Native protocol v4]
      Use HELP for help.
      cqlsh> CREATE KEYSPACE account WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };
      cqlsh> CREATE TABLE account.session  (
         ...   "token" blob,
         ...   account_id uuid,
         ...   PRIMARY KEY("token")
         ... )WITH compaction={'class': 'LeveledCompactionStrategy'} AND
         ...   compression={'sstable_compression': 'LZ4Compressor'};
      cqlsh> CREATE MATERIALIZED VIEW account.account_session AS
         ...        SELECT account_id,"token" FROM account.session
         ...        WHERE "token" IS NOT NULL and account_id IS NOT NULL
         ...        PRIMARY KEY (account_id, "token");
      ServerError: <ErrorMessage code=0000 [Server error] message="java.lang.RuntimeException: java.util.concurrent.ExecutionException: org.apache.cassandra.exceptions.SyntaxException: line 1:25 no viable alternative at input 'FROM' (SELECT account_id, token [FROM]...)">
      cqlsh> drop table account.session;
      ServerError: <ErrorMessage code=0000 [Server error] message="java.lang.RuntimeException: java.util.concurrent.ExecutionException: org.apache.cassandra.exceptions.SyntaxException: line 1:25 no viable alternative at input 'FROM' (SELECT account_id, token [FROM]...)">
      

      When any sstable*, nodetool, or when the Cassandra process is restarted, this is emitted on startup and Cassandra exits (copied from a server w/ data):

      INFO  [main] 2016-05-12 23:25:30,074 ColumnFamilyStore.java:395 - Initializing system_schema.indexes
      DEBUG [SSTableBatchOpen:1] 2016-05-12 23:25:30,075 SSTableReader.java:480 - Opening /mnt/cassandra/data/system_schema/indexes-0feb57ac311f382fba6d9024d305702f/ma-4-big (91 bytes)
      ERROR [main] 2016-05-12 23:25:30,143 CassandraDaemon.java:697 - Exception encountered during startup
      org.apache.cassandra.exceptions.SyntaxException: line 1:59 no viable alternative at input 'FROM' (..., expire_at, last_used, token [FROM]...)
              at org.apache.cassandra.cql3.ErrorCollector.throwFirstSyntaxError(ErrorCollector.java:101) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.cql3.CQLFragmentParser.parseAnyUnhandled(CQLFragmentParser.java:80) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.cql3.QueryProcessor.parseStatement(QueryProcessor.java:512) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.schema.SchemaKeyspace.fetchView(SchemaKeyspace.java:1128) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.schema.SchemaKeyspace.fetchViews(SchemaKeyspace.java:1092) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.schema.SchemaKeyspace.fetchKeyspace(SchemaKeyspace.java:903) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.schema.SchemaKeyspace.fetchKeyspacesWithout(SchemaKeyspace.java:879) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.schema.SchemaKeyspace.fetchNonSystemKeyspaces(SchemaKeyspace.java:867) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.config.Schema.loadFromDisk(Schema.java:134) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.config.Schema.loadFromDisk(Schema.java:124) ~[apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:229) [apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:551) [apache-cassandra-3.5.0.jar:3.5.0]
              at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:680) [apache-cassandra-3.5.0.jar:3.5.0]
      

      Attachments

        Issue Links

          Activity

            People

              carlyeks Carl Yeksigian
              victortrac Victor Trac
              Carl Yeksigian
              Alex Petrov
              Votes:
              0 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: