[
https://issues.apache.org/jira/browse/CASSANDRA-5351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13845937#comment-13845937
]
Lyuben Todorov commented on CASSANDRA-5351:
-------------------------------------------
[~jbellis] The weird thing is that although *newSSTables.size() !=
newSSTablesSize* the assertion doesn't actually cause an error, so no stack
trace, but swapping the assertion (as shown below) yields the stack trace
below.
{code}
if (newSSTables.size() != newSSTablesSize)
throw new RuntimeException(String.format("Expecting new size of %d, got %d
while replacing %s by %s in %s", newSSTablesSize, newSSTables.size(),
oldSSTables, replacements, this));
// assert newSSTables.size() == newSSTablesSize : String.format("Expecting new
size of %d, got %d while replacing %s by %s in %s", newSSTablesSize,
newSSTables.size(), oldSSTables, replacements, this);
{code}
Stack trace:
{code}
ERROR 00:50:24 Repair session 6265f5b0-62c7-11e3-ae2b-975f903ccf5a for range
(-9223372036854775808,-3074457345618258603] failed with error Expecting new
size of 3, got 4 while replacing
[SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-1-Data.db')]
by
[SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-10-Data.db'),
SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-11-Data.db')]
in View(pending_count=0,
sstables=[SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-8-Data.db'),
SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-9-Data.db')],
compacting=[])
java.lang.RuntimeException: Expecting new size of 3, got 4 while replacing
[SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-1-Data.db')]
by
[SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-10-Data.db'),
SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-11-Data.db')]
in View(pending_count=0,
sstables=[SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-8-Data.db'),
SSTableReader(path='/Users/me/.ccm/5351/node3/data/test/lvl/test-lvl-jc-9-Data.db')],
compacting=[])
at
org.apache.cassandra.db.DataTracker$View.newSSTables(DataTracker.java:631)
~[main/:na]
at
org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:587)
~[main/:na]
at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:378)
~[main/:na]
at
org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:252)
~[main/:na]
at
org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:1071)
~[main/:na]
at
org.apache.cassandra.db.compaction.CompactionManager.performAnticompaction(CompactionManager.java:295)
~[main/:na]
at
org.apache.cassandra.db.ColumnFamilyStore.forceAntiCompaction(ColumnFamilyStore.java:1047)
~[main/:na]
at
org.apache.cassandra.service.StorageService.anticompactRepairedRanges(StorageService.java:2565)
~[main/:na]
at
org.apache.cassandra.service.StorageService$5.runMayThrow(StorageService.java:2474)
~[main/:na]
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
~[main/:na]
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
~[na:1.7.0_25]
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
~[na:1.7.0_25]
at java.util.concurrent.FutureTask.run(FutureTask.java:166)
~[na:1.7.0_25]
at java.lang.Thread.run(Thread.java:724) ~[na:1.7.0_25]
{code}
*Note* I did check that I'm using the -ea parameter for the VM.
> Avoid repairing already-repaired data by default
> ------------------------------------------------
>
> Key: CASSANDRA-5351
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5351
> Project: Cassandra
> Issue Type: Task
> Components: Core
> Reporter: Jonathan Ellis
> Assignee: Lyuben Todorov
> Labels: repair
> Fix For: 2.1
>
>
> Repair has always built its merkle tree from all the data in a columnfamily,
> which is guaranteed to work but is inefficient.
> We can improve this by remembering which sstables have already been
> successfully repaired, and only repairing sstables new since the last repair.
> (This automatically makes CASSANDRA-3362 much less of a problem too.)
> The tricky part is, compaction will (if not taught otherwise) mix repaired
> data together with non-repaired. So we should segregate unrepaired sstables
> from the repaired ones.
--
This message was sent by Atlassian JIRA
(v6.1.4#6159)