Muir Manders created CASSANDRA-15077:
----------------------------------------

             Summary: Dropping column via thrift renders cf unreadable via CQL, 
leads to missing data
                 Key: CASSANDRA-15077
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15077
             Project: Cassandra
          Issue Type: Bug
            Reporter: Muir Manders


Hello

We have a lot of thrift/compact storage column families in production. We 
upgraded to 3.11.4 last week. This week we ran a (thrift) schema change to drop 
a column from a column family. Our CQL clients immediately starting getting a 
read error ("ReadFailure: Error from server: code=1300 ...") trying to read the 
column family. Thrift clients were still able to read the column family.

We determined restarting the nodes "fixed" CQL reads, so we did that, but soon 
discovered that we were missing data because cassandra was skipping sstables it 
didn't like on startup. That exception looked like this:
{noformat}
INFO  [main] 2019-04-04 20:06:35,676 ColumnFamilyStore.java:430 - Initializing 
test.test
ERROR [SSTableBatchOpen:1] 2019-04-04 20:06:35,689 CassandraDaemon.java:228 - 
Exception in thread Thread[SSTableBatchOpen:1,5,main]
java.lang.RuntimeException: Unknown column foo during deserialization
        at 
org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:326)
 ~[apache-cassandra-3.11.4.jar:3.11.4]
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522)
 ~[apache-cassandra-3.11.4.jar:3.11.4]
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385)
 ~[apache-cassandra-3.11.4.jar:3.11.4]
        at 
org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570)
 ~[apache-cassandra-3.11.4.jar:3.11.4]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_121]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[na:1.8.0_121]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) 
~[na:1.8.0_121]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
[na:1.8.0_121]
        at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
 [apache-cassandra-3.11.4.jar:3.11.4]
        at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_121]
{noformat}
 

Below is a list of steps to reproduce the issue. Note that in production our 
column families were all created via thrift, but I thought it was simpler to 
create them using CQL for the reproduction script.
{code}
ccm create test -v 3.11.4 -n 1
ccm updateconf 'start_rpc: true'
ccm start

sleep 10

ccm node1 cqlsh <<SCHEMA
CREATE KEYSPACE test WITH REPLICATION = {'class': 'SimpleStrategy', 
'replication_factor': 1};
CREATE COLUMNFAMILY test.test (
  id text,
  foo text,
  bar text,
  PRIMARY KEY (id)
) WITH COMPACT STORAGE;
INSERT INTO test.test (id, foo, bar) values ('1', 'hi', 'there');
SCHEMA

pip install pycassa

python <<DROP_COLUMN
import pycassa
sys = pycassa.system_manager.SystemManager('127.0.0.1:9160')
cf = sys.get_keyspace_column_families('test')['test']
sys.alter_column_family('test', 'test', column_metadata=filter(lambda c: c.name 
!= 'foo', cf.column_metadata))
DROP_COLUMN

# this produces the "ReadFailure: Error from server: code=1300" error
ccm node1 cqlsh <<QUERY
select * from test.test;
QUERY

ccm node1 stop
ccm node1 start

sleep 10

# this returns 0 rows (i.e. demonstrates missing data)
ccm node1 cqlsh <<QUERY
select * from test.test;
QUERY
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to