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

Node start can fail if the base table of a materialized view is not found

Agile BoardAttach filesAttach ScreenshotBulk Copy AttachmentsBulk Move AttachmentsVotersWatch issueWatchersCreate sub-taskConvert to sub-taskMoveLinkCloneLabelsUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Normal

    Description

      Node start can fail if the base table of a materialized view is not found, which is something that can happen under certain circumstances. There is a dtest reproducing the problem:

      cluster = self.cluster
      cluster.populate(3)
      cluster.start()
      node1, node2, node3 = self.cluster.nodelist()
      session = self.patient_cql_connection(node1, consistency_level=ConsistencyLevel.QUORUM)
      create_ks(session, 'ks', 3)
      
      session.execute('CREATE TABLE users (username varchar PRIMARY KEY, state varchar)')
      
      node3.stop(wait_other_notice=True)
      
      # create a materialized view only in nodes 1 and 2
      session.execute(('CREATE MATERIALIZED VIEW users_by_state AS '
                       'SELECT * FROM users WHERE state IS NOT NULL AND username IS NOT NULL '
                       'PRIMARY KEY (state, username)'))
      
      node1.stop(wait_other_notice=True)
      node2.stop(wait_other_notice=True)
      
      # drop the base table only in node 3
      node3.start(wait_for_binary_proto=True)
      session = self.patient_cql_connection(node3, consistency_level=ConsistencyLevel.QUORUM)
      session.execute('DROP TABLE ks.users')
      
      cluster.stop()
      cluster.start()  # Fails
      

      This is the error during node start:

      java.lang.IllegalArgumentException: Unknown CF 958ebc30-76e4-11e7-869a-9d8367a71c76
      	at org.apache.cassandra.db.Keyspace.getColumnFamilyStore(Keyspace.java:215) ~[main/:na]
      	at org.apache.cassandra.db.view.ViewManager.addView(ViewManager.java:143) ~[main/:na]
      	at org.apache.cassandra.db.view.ViewManager.reload(ViewManager.java:113) ~[main/:na]
      	at org.apache.cassandra.schema.Schema.alterKeyspace(Schema.java:618) ~[main/:na]
      	at org.apache.cassandra.schema.Schema.lambda$merge$18(Schema.java:591) ~[main/:na]
      	at java.util.Collections$UnmodifiableMap$UnmodifiableEntrySet.lambda$entryConsumer$0(Collections.java:1575) ~[na:1.8.0_131]
      	at java.util.HashMap$EntrySet.forEach(HashMap.java:1043) ~[na:1.8.0_131]
      	at java.util.Collections$UnmodifiableMap$UnmodifiableEntrySet.forEach(Collections.java:1580) ~[na:1.8.0_131]
      	at org.apache.cassandra.schema.Schema.merge(Schema.java:591) ~[main/:na]
      	at org.apache.cassandra.schema.Schema.mergeAndAnnounceVersion(Schema.java:564) ~[main/:na]
      	at org.apache.cassandra.schema.MigrationTask$1.response(MigrationTask.java:89) ~[main/:na]
      	at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:53) ~[main/:na]
      	at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) ~[main/:na]
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_131]
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_131]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_131]
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_131]
      	at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) [main/:na]
      	at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131]
      

      Attachments

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            adelapena Andres de la Peña Assign to me
            adelapena Andres de la Peña
            Andres de la Peña
            T Jake Luciani
            Votes:
            0 Vote for this issue
            Watchers:
            9 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment