cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Lerh Chuan Low (JIRA)" <j...@apache.org>
Subject [jira] [Created] (CASSANDRA-11943) >64k Clustering Keys cannot be flushed
Date Thu, 02 Jun 2016 01:25:59 GMT
Lerh Chuan Low created CASSANDRA-11943:
------------------------------------------

             Summary: >64k Clustering Keys cannot be flushed
                 Key: CASSANDRA-11943
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11943
             Project: Cassandra
          Issue Type: Bug
          Components: Local Write-Read Paths
            Reporter: Lerh Chuan Low


Setup:

I set this up with a 2 node cluster, but I think with a 1 node cluster it would encounter
the same issue. Use Cassandra 3.

{code}
CREATE KEYSPACE Blues WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor'
: 2};
CREATE TABLE test (a text, b text, PRIMARY KEY ((a), b))
{code}

Do the following insert:

{code}
CONSISTENCY ALL;
"INSERT INTO %s (a, b) VALUES ('foo', ?)", '<something that is bigger than 64k>')
{code}

Everything is fine and you can still run queries and so on, C* looks normal. But if we restart
C*, it never succeeds in starting up:

{code}
java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.AssertionError:
Attempted serializing to buffer exceeded maximum of 65535 bytes: 131082
	at org.apache.cassandra.utils.Throwables.maybeFail(Throwables.java:50) ~[main/:na]
	at org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:372) ~[main/:na]
	at org.apache.cassandra.db.commitlog.CommitLogReplayer.blockForWrites(CommitLogReplayer.java:257)
~[main/:na]
	at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:189) ~[main/:na]
	at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:168) ~[main/:na]
	at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:312) [main/:na]
	at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:583) [main/:na]
	at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:712) [main/:na]
Caused by: java.util.concurrent.ExecutionException: java.lang.AssertionError: Attempted serializing
to buffer exceeded maximum of 65535 bytes: 131082
	at java.util.concurrent.FutureTask.report(FutureTask.java:122) ~[na:1.8.0_40]
	at java.util.concurrent.FutureTask.get(FutureTask.java:192) ~[na:1.8.0_40]
	at org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:365) ~[main/:na]
	... 6 common frames omitted
Caused by: java.lang.AssertionError: Attempted serializing to buffer exceeded maximum of 65535
bytes: 131082
	at org.apache.cassandra.utils.ByteBufferUtil.writeWithShortLength(ByteBufferUtil.java:309)
~[main/:na]
	at org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.serialize(StatsMetadata.java:286)
~[main/:na]
	at org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.serialize(StatsMetadata.java:235)
~[main/:na]
	at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.serialize(MetadataSerializer.java:75)
~[main/:na]
	at org.apache.cassandra.io.sstable.format.big.BigTableWriter.writeMetadata(BigTableWriter.java:378)
~[main/:na]
	at org.apache.cassandra.io.sstable.format.big.BigTableWriter.access$300(BigTableWriter.java:51)
~[main/:na]
	at org.apache.cassandra.io.sstable.format.big.BigTableWriter$TransactionalProxy.doPrepare(BigTableWriter.java:342)
~[main/:na]
	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
~[main/:na]
	at org.apache.cassandra.io.sstable.format.SSTableWriter.prepareToCommit(SSTableWriter.java:280)
~[main/:na]
	at org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.prepareToCommit(SimpleSSTableMultiWriter.java:101)
~[main/:na]
	at org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1145)
~[main/:na]
	at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1095) ~[main/:na]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_40]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) ~[na:1.8.0_40]
	at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_40]
{code}

The same error as before can be reproduced if instead of restarting C* we call {{nodetool
flush}} after the insert, it looks like while flushing Memtables and attempting to serialize
{{SSTableMetadata}} it still expects CKeys less than 64k. 






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

Mime
View raw message