You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Cristian Marinescu (JIRA)" <ji...@apache.org> on 2014/11/10 09:36:33 UTC

[jira] [Created] (CASSANDRA-8280) Cassandra crashing on inserting data over 64K into indexed strings

Cristian Marinescu created CASSANDRA-8280:
---------------------------------------------

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


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=1000000)
        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 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) ~[na:1.7.0_60]
        at java.lang.Thread.run(Thread.java:745) ~[na:1.7.0_60]
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)