Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id C2461200B35 for ; Tue, 5 Jul 2016 14:18:12 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id C0C45160A6F; Tue, 5 Jul 2016 12:18:12 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 14B76160A4F for ; Tue, 5 Jul 2016 14:18:11 +0200 (CEST) Received: (qmail 75469 invoked by uid 500); 5 Jul 2016 12:18:11 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 75446 invoked by uid 99); 5 Jul 2016 12:18:11 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 05 Jul 2016 12:18:11 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 040E12C02A4 for ; Tue, 5 Jul 2016 12:18:11 +0000 (UTC) Date: Tue, 5 Jul 2016 12:18:11 +0000 (UTC) From: "Sylvain Lebresne (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CASSANDRA-11943) Allow value > 64K for clustering columns MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 05 Jul 2016 12:18:12 -0000 [ https://issues.apache.org/jira/browse/CASSANDRA-11943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15362410#comment-15362410 ] Sylvain Lebresne commented on CASSANDRA-11943: ---------------------------------------------- This is actually meant to be a follow up of CASSANDRA-11882. We probably validate that value > 64K are rejected for clustering values (since CASSANDRA-11882), but the fact we have this limitation in the first place (for 3.x) is only that we write the min/max clustering values in the sstable metadata and use the old serialization for that which limit values to 64K. So the goal of this ticket is to lift that limitation by rewriting the sstable metadata component without that limitation backed in. This is thus a 4.0 ticket which will require a sstable major version bump. > Allow value > 64K for clustering columns > ---------------------------------------- > > 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 > Assignee: Sylvain Lebresne > Fix For: 4.x > > > 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', ?)", '') > {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)