[
https://issues.apache.org/jira/browse/CASSANDRA-5256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14187189#comment-14187189
]
Nikolai Grigoriev commented on CASSANDRA-5256:
----------------------------------------------
I have just got this problem on multiple nodes. Cassandra 2.0.10 (DSE 4.5.2).
Should I reopen?
{code}
ERROR [CompactionExecutor:1196] 2014-10-28 17:14:50,124 CassandraDaemon.java
(line 199) Exception in thread Thread[CompactionExecutor:1196,1,main]
java.lang.AssertionError: Memory was freed
at org.apache.cassandra.io.util.Memory.checkPosition(Memory.java:259)
at org.apache.cassandra.io.util.Memory.getInt(Memory.java:211)
at
org.apache.cassandra.io.sstable.IndexSummary.getIndex(IndexSummary.java:79)
at
org.apache.cassandra.io.sstable.IndexSummary.getKey(IndexSummary.java:84)
at
org.apache.cassandra.io.sstable.IndexSummary.binarySearch(IndexSummary.java:58)
at
org.apache.cassandra.io.sstable.SSTableReader.getSampleIndexesForRanges(SSTableReader.java:692)
at
org.apache.cassandra.io.sstable.SSTableReader.estimatedKeysForRanges(SSTableReader.java:663)
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:328)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.findDroppableSSTable(LeveledCompactionStrategy.java:354)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getMaximalTask(LeveledCompactionStrategy.java:125)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:113)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:192)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}
> "Memory was freed" AssertionError During Major Compaction
> ---------------------------------------------------------
>
> Key: CASSANDRA-5256
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5256
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Affects Versions: 1.2.0
> Environment: Linux ashbdrytest01p 3.2.0-37-generic #58-Ubuntu SMP Thu
> Jan 24 15:28:10 UTC 2013 x86_64 x86_64 x86_64 GNU/Linux
> java version "1.6.0_30"
> Java(TM) SE Runtime Environment (build 1.6.0_30-b12)
> Java HotSpot(TM) 64-Bit Server VM (build 20.5-b03, mixed mode)
> Ubuntu 12.04.2 LTS
> Reporter: C. Scott Andreas
> Assignee: Jonathan Ellis
> Priority: Critical
> Labels: compaction
> Fix For: 1.2.2
>
> Attachments: 5256-v2.txt, 5256-v4.txt, 5256-v5.txt, 5256.txt
>
>
> When initiating a major compaction with `./nodetool -h localhost compact`, an
> AssertionError is thrown in the CompactionExecutor from o.a.c.io.util.Memory:
> ERROR [CompactionExecutor:41495] 2013-02-14 14:38:35,720 CassandraDaemon.java
> (line 133) Exception in thread Thread[CompactionExecutor:41495,1,RMI Runtime]
> java.lang.AssertionError: Memory was freed
> at org.apache.cassandra.io.util.Memory.checkPosition(Memory.java:146)
> at org.apache.cassandra.io.util.Memory.getLong(Memory.java:116)
> at
> org.apache.cassandra.io.compress.CompressionMetadata.chunkFor(CompressionMetadata.java:176)
> at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:88)
> at
> org.apache.cassandra.io.util.RandomAccessReader.read(RandomAccessReader.java:327)
> at java.io.RandomAccessFile.readInt(RandomAccessFile.java:755)
> at java.io.RandomAccessFile.readLong(RandomAccessFile.java:792)
> at
> org.apache.cassandra.utils.BytesReadTracker.readLong(BytesReadTracker.java:114)
> at
> org.apache.cassandra.db.ColumnSerializer.deserializeColumnBody(ColumnSerializer.java:101)
> at
> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:92)
> at
> org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumnsFromSSTable(ColumnFamilySerializer.java:149)
> at
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWithColumns(SSTableIdentityIterator.java:235)
> at
> org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:109)
> at
> org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:93)
> at
> org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:162)
> at
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:76)
> at
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:57)
> at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:114)
> at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:97)
> at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> at
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:158)
> at
> org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:71)
> at
> org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:342)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)
> ---
> I've invoked the `nodetool compact` three times; this occurred after each.
> The node has been up for a couple days accepting writes and has not been
> restarted.
> Here's the server's log since it was started a few days ago:
> https://gist.github.com/cscotta/4956472/raw/95e7cbc68de1aefaeca11812cbb98d5d46f534e8/cassandra.log
> Here's the code being used to issue writes to the datastore:
> https://gist.github.com/cscotta/20cbd36c2503c71d06e9
> ---
> Configuration: One node, one keyspace, one column family. ~60 writes/second
> of data with a TTL of 86400, zero reads. Stock cassandra.yaml.
> Keyspace DDL:
> create keyspace jetpack;
> use jetpack;
> create column family Metrics with key_validation_class = 'UTF8Type' and
> comparator = 'IntegerType';
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)