[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14221987#comment-14221987 ] Sam Tunnicliffe commented on CASSANDRA-8280: Unfortunately, that check is required as CompositeType.Builder.build() uses ByteBufferUtil.writeWithShortLength so we never reach ThriftValidation.validateKey(cfm, key) Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 Attachments: 8280-2.0-v2.txt, 8280-2.0-v3.txt, 8280-2.0.txt, 8280-2.1-v2.txt, 8280-2.1.txt An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.1.jar:2.1.1] at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.jar:na] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14222084#comment-14222084 ] Aleksey Yeschenko commented on CASSANDRA-8280: -- Let me clarify. I meant we should do `keyBuilder.copy().add(val)` unconditionally, but only call the final `build()` (that used `ByteBufferUtil.writeWithShortLength()` if getLength() validates. And if we put our check there, then we no longer need the `ThriftValidation.validateKey()` call at all. And the subsequent call to `ThriftValidation.validateKey()` in `ModificationStatement#getMutations()`, either. Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 Attachments: 8280-2.0-v2.txt, 8280-2.0-v3.txt, 8280-2.0.txt, 8280-2.1-v2.txt, 8280-2.1.txt An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14222138#comment-14222138 ] Sam Tunnicliffe commented on CASSANDRA-8280: Well, ThriftValidation.validateKey() does more than simply check the length, so I think we'd still need it. Also, we do potentially less work the way it is now because we only iterate over the builder components (in getLength()) once, rather than once per value. Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 Attachments: 8280-2.0-v2.txt, 8280-2.0-v3.txt, 8280-2.0.txt, 8280-2.1-v2.txt, 8280-2.1.txt An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.1.jar:2.1.1] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14222191#comment-14222191 ] Aleksey Yeschenko commented on CASSANDRA-8280: -- Fair enough re: TV.validateKey() doing more than that (although the second call, in MS#getMutations(), is still redundant, but that's not an issue introduced by this patch). In that case you'd need logic more complex than `baseSize + val.remaining()`, to account for extra 3 bytes, conditionally, for partition keys. I'd rather we did some extra iteration (which in this case happens rarely and doesn't cost much, vs. extra complexity). Anyway, sorry, this is me extremely nit-picking for lack of things to do on Saturday. Feel free to ignore the minor things. Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 Attachments: 8280-2.0-v2.txt, 8280-2.0-v3.txt, 8280-2.0.txt, 8280-2.1-v2.txt, 8280-2.1.txt An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14220710#comment-14220710 ] Sam Tunnicliffe commented on CASSANDRA-8280: My original patches didn't cover conditional updates, so I've attached v2 of both. Also, because there's no easy way to execute conditional CQL in a unit test I've written a dtest which is included in [this|https://github.com/riptano/cassandra-dtest/pull/119] pull request. The dtest kind of makes the unit test redundant, but it felt wrong to remove it so I've left it in the patches. I should also note that in cases where the indexed column is a clustering component or partition key, we would already reject update with these oversized values. In 2.0 though, this results in an ServerError rather than an InvalidRequest so I've added a little more validation to return a nicer error response. Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 Attachments: 8280-2.0-v2.txt, 8280-2.0.txt, 8280-2.1-v2.txt, 8280-2.1.txt An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14220948#comment-14220948 ] Aleksey Yeschenko commented on CASSANDRA-8280: -- LGTM, with one last 2.0 nit - can you put the builder total size calculation logic into a method in ColumnNameBuilder? That, and you should include the extra 3 bytes per component in the calculation (2 for size, one for EOC). Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 Attachments: 8280-2.0-v2.txt, 8280-2.0.txt, 8280-2.1-v2.txt, 8280-2.1.txt An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.1.jar:2.1.1] at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.jar:na]
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14221294#comment-14221294 ] Aleksey Yeschenko commented on CASSANDRA-8280: -- Looks to me like you don't need the partition key length check in ModificationStatement - the subsequent call to ThriftValidation.validateKey(cfm, key) will validate the length. Also, should override addUpdateForKey() in DeleteStatement, too. Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 Attachments: 8280-2.0-v2.txt, 8280-2.0-v3.txt, 8280-2.0.txt, 8280-2.1-v2.txt, 8280-2.1.txt An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.1.jar:2.1.1] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14214634#comment-14214634 ] Sam Tunnicliffe commented on CASSANDRA-8280: There is already validation for this on the thrift path (from CASSANDRA-3057), but it appears to have been overlooked for CQL. Attaching separate patches for 2.0 2.1, mainly because CQLTester in 2.1 makes the unit test simpler but not easily mergeable. Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Assignee: Sam Tunnicliffe Priority: Critical Fix For: 2.1.3 An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.1.jar:2.1.1] at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.jar:na] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14204749#comment-14204749 ] Michael Shuler commented on CASSANDRA-8280: --- Thanks for the bug report - reproduced in 2.1.2 and current 2.1 branch HEAD. Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Priority: Critical Fix For: 2.1.3 An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.1.jar:2.1.1] at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.jar:na] at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1053) ~[apache-cassandra-2.1.1.jar:2.1.1] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_60] at
[jira] [Commented] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings
[ https://issues.apache.org/jira/browse/CASSANDRA-8280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14204772#comment-14204772 ] Michael Shuler commented on CASSANDRA-8280: --- current ByteBufferUtil.java:290 was committed in CASSANDRA-6781 Cassandra crashing on inserting data over 64K into indexed strings -- Key: CASSANDRA-8280 URL: https://issues.apache.org/jira/browse/CASSANDRA-8280 Project: Cassandra Issue Type: Bug Components: Core Environment: Debian 7, Cassandra 2.1.1, java 1.7.0_60 Reporter: Cristian Marinescu Priority: Critical Fix For: 2.1.3 An attemtp to instert 65536 bytes in a field that is a primary index throws (correctly?) the cassandra.InvalidRequest exception. However, inserting the same data *in a indexed field that is not a primary index* works just fine. However, Cassandra will crash on next commit and never recover. So I rated it as Critical as it can be used for DoS attacks. Reproduce: see the snippet below: {code} import uuid from cassandra import ConsistencyLevel from cassandra import InvalidRequest from cassandra.cluster import Cluster from cassandra.auth import PlainTextAuthProvider from cassandra.policies import ConstantReconnectionPolicy from cassandra.cqltypes import UUID # DROP KEYSPACE IF EXISTS cs; # CREATE KEYSPACE cs WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; # USE cs; # CREATE TABLE test3 (name text, value uuid, sentinel text, PRIMARY KEY (name)); # CREATE INDEX test3_sentinels ON test3(sentinel); class CassandraDemo(object): def __init__(self): ips = [127.0.0.1] ap = PlainTextAuthProvider(username=cs, password=cs) reconnection_policy = ConstantReconnectionPolicy(20.0, max_attempts=100) cluster = Cluster(ips, auth_provider=ap, protocol_version=3, reconnection_policy=reconnection_policy) self.session = cluster.connect(cs) def exec_query(self, query, args): prepared_statement = self.session.prepare(query) prepared_statement.consistency_level = ConsistencyLevel.LOCAL_QUORUM self.session.execute(prepared_statement, args) def bug(self): k1 = UUID( str(uuid.uuid4()) ) long_string = X * 65536 query = INSERT INTO test3 (name, value, sentinel) VALUES (?, ?, ?); args = (foo, k1, long_string) self.exec_query(query, args) self.session.execute(DROP KEYSPACE IF EXISTS cs_test, timeout=30) self.session.execute(CREATE KEYSPACE cs_test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}) c = CassandraDemo() #first run c.bug() #second run, Cassandra crashes with java.lang.AssertionError c.bug() {code} And here is the cassandra log: {code} ERROR [MemtableFlushWriter:3] 2014-11-06 16:44:49,263 CassandraDaemon.java:153 - Exception in thread Thread[MemtableFlushWriter:3,5,main] java.lang.AssertionError: 65536 at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:290) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.maybeWriteRowHeader(ColumnIndex.java:214) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:201) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:142) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:354) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:312) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.1.jar:2.1.1] at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.jar:na] at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1053) ~[apache-cassandra-2.1.1.jar:2.1.1] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_60] at