cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Stefania (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-13897) nodetool compact and flush fail with "error: null"
Date Wed, 18 Oct 2017 08:45:00 GMT

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

Stefania commented on CASSANDRA-13897:
--------------------------------------

[~blambov], can the {{CachingRebufferer}} be changed to only require a multiple of 4096 rather
than a power of two? [~snazy] and myself are worried that neither rounding up nor rounding
down to a power of two feels too right as in both cases we may end up being quite far from
the initial value derived from the data and the disk optimization strategy. As far as I have
understood, the rebufferer requires a power of two to calculate the page position, can it
not align to 4096 multiples instead?

> nodetool compact and flush fail with "error: null"
> --------------------------------------------------
>
>                 Key: CASSANDRA-13897
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13897
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Compaction
>         Environment: * Apache Cassandra 3.11.0
> * Linux 4.4.0-92-generic #115-Ubuntu SMP Thu Aug 10 09:04:33 UTC 2017 x86_64 x86_64 x86_64
GNU/Linux
> * jdk1.8.0_144
>            Reporter: Jacob Rhoden
>            Assignee: Stefania
>            Priority: Minor
>
> {{nodetool flush}} and {{nodetool compact}} return an error message that is not clear.
This could probably be improved. Both of my two nodes return this error.
> {{nodetool flush}} Will return this error the first 2-3 times you invoke it, then the
error temporarily disappears. {{nodetool compress}} always returns this error message no matter
how many times you invoke it.
> I have tried deleting saved_caches, commit logs, doing nodetool compact/rebuild/scrub,
and nothing seems to remove the error. 
> {noformat}
> cass@s5:~/apache-cassandra-3.11.0$ nodetool compact
> error: null
> -- StackTrace --
> java.lang.AssertionError
> 	at org.apache.cassandra.cache.ChunkCache$CachingRebufferer.<init>(ChunkCache.java:222)
> 	at org.apache.cassandra.cache.ChunkCache.wrap(ChunkCache.java:175)
> 	at org.apache.cassandra.io.util.FileHandle$Builder.maybeCached(FileHandle.java:412)
> 	at org.apache.cassandra.io.util.FileHandle$Builder.complete(FileHandle.java:381)
> 	at org.apache.cassandra.io.util.FileHandle$Builder.complete(FileHandle.java:331)
> 	at org.apache.cassandra.io.sstable.format.big.BigTableWriter.openFinal(BigTableWriter.java:333)
> 	at org.apache.cassandra.io.sstable.format.big.BigTableWriter.openFinalEarly(BigTableWriter.java:318)
> 	at org.apache.cassandra.io.sstable.SSTableRewriter.switchWriter(SSTableRewriter.java:322)
> 	at org.apache.cassandra.io.sstable.SSTableRewriter.doPrepare(SSTableRewriter.java:370)
> 	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
> 	at org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.doPrepare(CompactionAwareWriter.java:111)
> 	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
> 	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.finish(Transactional.java:184)
> 	at org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.finish(CompactionAwareWriter.java:121)
> 	at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:220)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> 	at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:85)
> 	at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
> 	at org.apache.cassandra.db.compaction.CompactionManager$10.runMayThrow(CompactionManager.java:733)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	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:81)
> 	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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message