[ 
https://issues.apache.org/jira/browse/CASSANDRA-13973?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16216074#comment-16216074
 ] 

Jeff Jirsa edited comment on CASSANDRA-13973 at 10/23/17 11:59 PM:
-------------------------------------------------------------------

Those knobs are actually for a slightly different index (the partition summary 
itself). Though in yaml / {{Config.java}} there's {{column_index_size_in_kb}} 
(defaults to 64), which I'm *pretty sure* does this, but not sure enough to 
tell you to change it without testing in a lab (in particular, I don't know 
exactly how it behaves if you change this with pre-existing data; I'd hope it'd 
do the right thing, but I certainly haven't personally tested it, so *this is 
not a recommendation* , only a pointer to a config option that exists). In 
fact, CASSANDRA-5454 notes that this used to cause corruption, so I'm 
definitely not saying you should do this, though it's possible that 
CASSANDRA-5454 makes this safe to do. 

(FWIW, on the index summary, we downsample if we get over 2GB - see 
CASSANDRA-12014 -  and I think instead of changing this to a long, we should 
consider that here as well, until CASSANDRA-9754 is done, which makes this a 
moot point). 

[~krummas] / [~bdeggleston] , what do you guys think about the safety of this, 
especially re: streaming? 


was (Author: jjirsa):
Those knobs are actually for a slightly different index (the partition summary 
itself). Though in yaml / {{Config.java}} there's {{column_index_size_in_kb}} 
(defaults to 64), which I'm *pretty sure* does this, but not sure enough to 
tell you to change it without testing in a lab (in particular, I don't know 
exactly how it behaves if you change this with pre-existing data; I'd hope it'd 
do the right thing, but I certainly haven't personally tested it, so *this is 
not a recommendation* , only a pointer to a config option that exists). In 
fact, CASSANDRA-5454 notes that this used to cause corruption, so I'm 
definitely not saying you should do this, though it's possible that 
CASSANDRA-5454 makes this safe to do. 

(FWIW, on the index summary, we downsample if we get over 2GB - see 
CASSANDRA-12014 -  and I think instead of changing this to a long, we should 
consider that here as well, until CASSANDRA-9754 is done, which makes this a 
moot point)

> IllegalArgumentException in upgradesstables compaction
> ------------------------------------------------------
>
>                 Key: CASSANDRA-13973
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13973
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Compaction
>            Reporter: Dan Kinder
>
> After an upgrade from 2.2.6 to 3.0.15 (sstable version la to mc), when I try 
> to run upgradesstables, most of them upgrade fine but I see the exception 
> below on several nodes, and it doesn't complete.
> CASSANDRA-12717 looks similar but the stack trace is not the same, so I 
> assumed it is not identical. The various nodes this happens on all give the 
> same trace.
> Might be notable that this is an analytics cluster with some large 
> partitions, in the GB size.
> {noformat}
> error: Out of range: 7316844981
> -- StackTrace --
> java.lang.IllegalArgumentException: Out of range: 7316844981
> at com.google.common.primitives.Ints.checkedCast(Ints.java:91)
> at 
> org.apache.cassandra.db.RowIndexEntry$IndexedEntry.promotedSize(RowIndexEntry.java:329)
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.serialize(RowIndexEntry.java:133)
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter$IndexWriter.append(BigTableWriter.java:409)
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.afterAppend(BigTableWriter.java:120)
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.append(BigTableWriter.java:157)
> at 
> org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:125)
> at 
> org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter.realAppend(MaxSSTableSizeWriter.java:88)
> at 
> org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.append(CompactionAwareWriter.java:109)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:195)
> at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:89)
> at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$5.execute(CompactionManager.java:424)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:311)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> at java.lang.Thread.run(Thread.java:748)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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

Reply via email to