[
https://issues.apache.org/jira/browse/CASSANDRA-2261?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13193195#comment-13193195
]
Pavel Yaskevich commented on CASSANDRA-2261:
--------------------------------------------
Patch overall looks good, has some code styling issues in files
(LeveledManifest.java, SizeTieredCompactionStrategy.java).
And when you comment logger.debug(...) in ColumnFamilyStore.java:1332 you will
be able to see following exception (one exception for each of the
Keyspace1-Standard1 SSTables) shown by CompactionsTest.testBlacklisting():
{nofromat}
[junit] java.lang.RuntimeException:
SSTableScanner(file=RandomAccessReader(filePath='/Users/xedin/work/java/cassandra/build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hc-5-Data.db',
skipIOCache=true)
sstable=SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hc-5-Data.db')
exhausted=false) failed to provide next columns from
KeyScanningIterator(finishedAt:0)
[junit] at
org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:193)
[junit] at
org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:146)
[junit] at
org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:138)
[junit] at
org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38)
[junit] at
org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:149)
[junit] at
org.apache.cassandra.utils.MergeIterator$ManyToOne.<init>(MergeIterator.java:90)
[junit] at
org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:47)
[junit] at
org.apache.cassandra.db.compaction.CompactionIterable.iterator(CompactionIterable.java:79)
[junit] at
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:129)
[junit] at
org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:260)
[junit] at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
[junit] at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
[junit] at
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
[junit] at java.util.concurrent.FutureTask.run(FutureTask.java:138)
[junit] at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
[junit] at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
[junit] at java.lang.Thread.run(Thread.java:680)
[junit] Caused by: java.io.EOFException
[junit] at java.io.RandomAccessFile.readFully(RandomAccessFile.java:399)
[junit] at java.io.RandomAccessFile.readFully(RandomAccessFile.java:377)
[junit] at
org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:324)
[junit] at
org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:391)
[junit] at
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:373)
[junit] at
org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:173)
[junit] ... 16 more
{noformat}
Could you please investigate that cases them?
> During Compaction, Corrupt SSTables with rows that cause failures should be
> identified and blacklisted.
> -------------------------------------------------------------------------------------------------------
>
> Key: CASSANDRA-2261
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2261
> Project: Cassandra
> Issue Type: Improvement
> Components: Core
> Reporter: Benjamin Coverston
> Assignee: Benjamin Coverston
> Priority: Minor
> Labels: not_a_pony
> Fix For: 1.1
>
> Attachments: 2261-v2.patch, 2261.patch
>
>
> When a compaction of a set of SSTables fails because of corruption it will
> continue to try to compact that SSTable causing pending compactions to build
> up.
> One way to mitigate this problem would be to log the error, then identify the
> specific SSTable that caused the failure, subsequently blacklisting that
> SSTable and ensuring that it is no longer included in future compactions. For
> this we could simply store the problematic SSTable's name in memory.
> If it's not possible to identify the SSTable that caused the issue, then
> perhaps blacklisting the (ordered) permutation of SSTables to be compacted
> together is something that can be done to solve this problem in a more
> general case, and avoid issues where two (or more) SSTables have trouble
> compacting a particular row. For this option we would probably want to store
> the lists of the bad combinations in the system table somewhere s.t. these
> can survive a node failure (there have been a few cases where I have seen a
> compaction cause a node failure).
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira