[ 
https://issues.apache.org/jira/browse/CASSANDRA-6375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne updated CASSANDRA-6375:
----------------------------------------

    Attachment: 0002-TMD-ConcurrentModificationException.txt

Regarding the ConcurrentModificationException, it seems that the only reason 
this could get triggered is due to TMD.clearUnsafe(). As this is called by 
tests, this is not a real problem, but what about making it grab the writeLock 
like any good citizen to avoid getting scarry stack traces (and don't discard a 
real bug later on because we've grown used to discarding such stack)? Attaching 
patch to do that.

> Unit test failures on 1.2 branch
> --------------------------------
>
>                 Key: CASSANDRA-6375
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6375
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Sylvain Lebresne
>            Priority: Blocker
>             Fix For: 1.2.12
>
>         Attachments: 0001-LeaveAndBootstrapTest.txt, 
> 0002-TMD-ConcurrentModificationException.txt
>
>
> On my box, I get a number of reproducible test failures:
> # LeaveAndBootstrapTest
> {noformat}
> [junit] Testsuite: org.apache.cassandra.service.LeaveAndBootstrapTest
> [junit] Tests run: 6, Failures: 2, Errors: 0, Time elapsed: 13.145 sec
> [junit] 
> [junit] ------------- Standard Error -----------------
> [junit]  WARN 12:11:43,275 Node /127.0.0.3 'leaving' token mismatch. Long 
> network partition?
> [junit] ------------- ---------------- ---------------
> [junit] Testcase: 
> newTestWriteEndpointsDuringLeave(org.apache.cassandra.service.LeaveAndBootstrapTest):
>        FAILED
> [junit] mismatched endpoint sets expected:<[/127.0.0.4, /127.0.0.5]> but 
> was:<[/127.0.0.4]>
> [junit] junit.framework.AssertionFailedError: mismatched endpoint sets 
> expected:<[/127.0.0.4, /127.0.0.5]> but was:<[/127.0.0.4]>
> [junit]       at 
> org.apache.cassandra.service.LeaveAndBootstrapTest.newTestWriteEndpointsDuringLeave(LeaveAndBootstrapTest.java:131)
> {noformat}
> # TokenMetadataTest
> {noformat}
> [junit] Testsuite: org.apache.cassandra.locator.TokenMetadataTest
> [junit] Tests run: 3, Failures: 1, Errors: 0, Time elapsed: 0.76 sec
> [junit] 
> [junit] Testcase: 
> testRingIterator(org.apache.cassandra.locator.TokenMetadataTest):   FAILED
> [junit] [] expected:<2> but was:<0>
> [junit] junit.framework.AssertionFailedError: [] expected:<2> but was:<0>
> [junit]       at 
> org.apache.cassandra.locator.TokenMetadataTest.testRingIterator(TokenMetadataTest.java:55)
> [junit]       at 
> org.apache.cassandra.locator.TokenMetadataTest.testRingIterator(TokenMetadataTest.java:63)
> {noformat}
> # ScrubTest
> {noformat}
> [junit] Testsuite: org.apache.cassandra.db.ScrubTest
> [junit] Tests run: 4, Failures: 1, Errors: 0, Time elapsed: 12.499 sec
> [junit] 
> [junit] ------------- Standard Error -----------------
> [junit]  WARN 12:16:27,799 Out of order row detected (DecoratedKey(63, 63) 
> found after DecoratedKey(7a, 7a))
> [junit]  WARN 12:16:27,801 Out of order row detected (DecoratedKey(79, 79) 
> found after DecoratedKey(7a, 7a))
> [junit]  WARN 12:16:27,802 Out of order row detected (DecoratedKey(64, 64) 
> found after DecoratedKey(7a, 7a))
> [junit]  WARN 12:16:28,289 3 out of order rows found while scrubbing 
> SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard3/Keyspace1-Standard3-ia-1-Data.db');
>  Those have been written (in order) to a new sstable 
> (SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard3/Keyspace1-Standard3-ic-3-Data.db'))
> [junit] ------------- ---------------- ---------------
> [junit] Testcase: testScrubOneRow(org.apache.cassandra.db.ScrubTest): FAILED
> [junit] expected:<1> but was:<10>
> [junit] junit.framework.AssertionFailedError: expected:<1> but was:<10>
> [junit]       at 
> org.apache.cassandra.db.ScrubTest.testScrubOneRow(ScrubTest.java:94)
> {noformat}
> While running the whole test suites I also ran into the following stack:
> {noformat}
> [junit] Testsuite: org.apache.cassandra.dht.BootStrapperTest
> [junit] Tests run: 4, Failures: 0, Errors: 0, Time elapsed: 14.345 sec
> [junit] 
> [junit] ------------- Standard Error -----------------
> [junit]  WARN 11:16:50,833 No host ID found, created 
> cb1c4ca1-c451-42ae-b205-36258dfe4f96 (Note: This should happen exactly once 
> per node).
> [junit]  WARN 11:16:51,193 Generated random token 
> [f368755beab4290b7e70895776c6e14e]. Random tokens will result in an 
> unbalanced ring; see http://wiki.apache.org/cassandra/Operations
> [junit] ERROR 11:16:51,724 Fatal exception in thread 
> Thread[PendingRangeCalculator:1,5,main]
> [junit] java.util.ConcurrentModificationException
> [junit]       at 
> java.util.TreeMap$PrivateEntryIterator.nextEntry(TreeMap.java:1115)
> [junit]       at java.util.TreeMap$EntryIterator.next(TreeMap.java:1151)
> [junit]       at java.util.TreeMap$EntryIterator.next(TreeMap.java:1146)
> [junit]       at 
> com.google.common.collect.AbstractMultimap$EntryIterator.findValueIteratorAndKey(AbstractMultimap.java:1152)
> [junit]       at 
> com.google.common.collect.AbstractMultimap$EntryIterator.next(AbstractMultimap.java:1166)
> [junit]       at 
> com.google.common.collect.AbstractMultimap$EntryIterator.next(AbstractMultimap.java:1136)
> [junit]       at 
> java.util.Collections$UnmodifiableCollection$1.next(Collections.java:1067)
> [junit]       at 
> com.google.common.collect.ForwardingIterator.next(ForwardingIterator.java:48)
> [junit]       at 
> com.google.common.collect.Maps$UnmodifiableEntries$1.next(Maps.java:953)
> [junit]       at 
> com.google.common.collect.Maps$UnmodifiableEntries$1.next(Maps.java:951)
> [junit]       at 
> com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:272)
> [junit]       at 
> com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
> [junit]       at 
> org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
> [junit]       at 
> org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
> [junit]       at 
> org.apache.cassandra.locator.TokenMetadata.cloneAfterAllLeft(TokenMetadata.java:619)
> [junit]       at 
> org.apache.cassandra.service.PendingRangeCalculatorService.calculatePendingRanges(PendingRangeCalculatorService.java:139)
> [junit]       at 
> org.apache.cassandra.service.PendingRangeCalculatorService$PendingRangeTask.run(PendingRangeCalculatorService.java:67)
> [junit]       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> [junit]       at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> [junit]       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> [junit]       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> [junit]       at java.lang.Thread.run(Thread.java:744)
> {noformat}
> This doesn't end up failing the test and I was actually not able to reproduce 
> when running BootStrapperTest individually, but I don't know if we understand 
> why that can happen during the test (and if it's just an artifact of testing 
> or a real thing).



--
This message was sent by Atlassian JIRA
(v6.1#6144)

Reply via email to