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

Joshua McKenzie commented on CASSANDRA-9104:
--------------------------------------------

Branch available 
[here|https://github.com/apache/cassandra/compare/trunk...josh-mckenzie:9104_v3].
 The new errors were caused by SchemaLoader.prepareServer setting the 
createReserveSegments flag to true manually after a cleanupAndLeaveDirs call, 
leading to the failures during the Embedded cassandra server's startup on cql 
tests. Unfortunately we can't just remove that and move on as this illustrates 
a problem: we have 2 different use-cases for SchemaLoader - tests that run with 
the state provided by the base-class and tests that run with an 
EmbeddedCassandraService. If we don't enable reserve segment creation, the 
former will hang waiting for allocation that never gets prompted. If we enable 
reserve segment creation, the latter will fail with the new assertion we've 
added.

Rather than break the contract that the SchemaLoader will always leave the 
CommitLog in an allocated and operating state and track down all the other unit 
tests that breaks, I've chosen the route of making CL.recover idempotent and to 
simply return 0 on a recover call when the allocator's createReserveSegment 
flag is already true, indicating recovery has already taken place. It's 
important to note that this is only a circumstance that arises during unit 
tests. I've also made CLSM.enableReserveSegmentCreation package private to 
discourage getting the allocator into a bad state (as well as remove the need 
to manually manage it) and slightly restructured CL.startUnsafe() so an unsafe 
start will always recover and leave the CL and CLSM in a ready-state for tests. 
In my opinion, the least-surprising thing to do here is to allow the startup of 
an EmbeddedCassandraService if the CL is in a normal operational state from a 
previous test.

I've run utests on both windows and linux and this leaves us with a 2-3 timeout 
failures on Windows and 1 timeout on linux - 
IndexSummaryManagerTest:testCompactionRace.

> Unit test failures, trunk + Windows
> -----------------------------------
>
>                 Key: CASSANDRA-9104
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-9104
>             Project: Cassandra
>          Issue Type: Test
>            Reporter: Joshua McKenzie
>            Assignee: Joshua McKenzie
>              Labels: Windows
>             Fix For: 3.x
>
>         Attachments: 9104_CFSTest.txt, 9104_KeyCache.txt, 
> 9104_KeyCache_ScrubTest_v2.txt, 9104_RecoveryManager.txt, 
> 9104_RecoveryManager_v2.txt, 9104_ScrubTest.txt
>
>
> Variety of different test failures have cropped up over the past 2-3 weeks:
> h6. -org.apache.cassandra.cql3.UFTest FAILED (timeout)- // No longer failing 
> / timing out
> h6. 
> testLoadNewSSTablesAvoidsOverwrites(org.apache.cassandra.db.ColumnFamilyStoreTest):
>        FAILED
> {noformat}
>    12 SSTables unexpectedly exist
>    junit.framework.AssertionFailedError: 12 SSTables unexpectedly exist
>    at 
> org.apache.cassandra.db.ColumnFamilyStoreTest.testLoadNewSSTablesAvoidsOverwrites(ColumnFamilyStoreTest.java:1896)
> {noformat}
> h6. org.apache.cassandra.db.KeyCacheTest FAILED
> {noformat}
>    expected:<4> but was:<2>
>    junit.framework.AssertionFailedError: expected:<4> but was:<2>
>    at 
> org.apache.cassandra.db.KeyCacheTest.assertKeyCacheSize(KeyCacheTest.java:221)
>    at org.apache.cassandra.db.KeyCacheTest.testKeyCache(KeyCacheTest.java:181)
> {noformat}
> h6. RecoveryManagerTest:
> {noformat}
>    org.apache.cassandra.db.RecoveryManagerTest FAILED
>    org.apache.cassandra.db.RecoveryManager2Test FAILED
>    org.apache.cassandra.db.RecoveryManager3Test FAILED
>    org.apache.cassandra.db.RecoveryManagerTruncateTest FAILED
>    All are the following:
>       java.nio.file.AccessDeniedException: 
> build\test\cassandra\commitlog;0\CommitLog-5-1427995105229.log
>       FSWriteError in 
> build\test\cassandra\commitlog;0\CommitLog-5-1427995105229.log
>          at 
> org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:128)
>          at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.recycleSegment(CommitLogSegmentManager.java:360)
>          at 
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:156)
>          at 
> org.apache.cassandra.db.RecoveryManagerTest.testNothingToRecover(RecoveryManagerTest.java:75)
>       Caused by: java.nio.file.AccessDeniedException: 
> build\test\cassandra\commitlog;0\CommitLog-5-1427995105229.log
>          at 
> sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:83)
>          at 
> sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
>          at 
> sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102)
>          at 
> sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:269)
>          at 
> sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSystemProvider.java:103)
>          at java.nio.file.Files.delete(Files.java:1079)
>          at 
> org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:124)
> {noformat}
> h6. testScrubCorruptedCounterRow(org.apache.cassandra.db.ScrubTest):  FAILED
> {noformat}
> Expecting new size of 1, got 2 while replacing 
> [BigTableReader(path='C:\src\refCassandra\build\test\cassandra\data;0\Keyspace1\Counter1-deab62b2d95c11e489c6e117fe147c1d\la-1-big-Data.db')]
>  by 
> [BigTableReader(path='C:\src\refCassandra\build\test\cassandra\data;0\Keyspace1\Counter1-deab62b2d95c11e489c6e117fe147c1d\la-1-big-Data.db')]
>  in View(pending_count=0, 
> sstables=[BigTableReader(path='C:\src\refCassandra\build\test\cassandra\data;0\Keyspace1\Counter1-deab62b2d95c11e489c6e117fe147c1d\la-3-big-Data.db')],
>  compacting=[])
> junit.framework.AssertionFailedError: Expecting new size of 1, got 2 while 
> replacing 
> [BigTableReader(path='C:\src\refCassandra\build\test\cassandra\data;0\Keyspace1\Counter1-deab62b2d95c11e489c6e117fe147c1d\la-1-big-Data.db')]
>  by 
> [BigTableReader(path='C:\src\refCassandra\build\test\cassandra\data;0\Keyspace1\Counter1-deab62b2d95c11e489c6e117fe147c1d\la-1-big-Data.db')]
>  in View(pending_count=0, 
> sstables=[BigTableReader(path='C:\src\refCassandra\build\test\cassandra\data;0\Keyspace1\Counter1-deab62b2d95c11e489c6e117fe147c1d\la-3-big-Data.db')],
>  compacting=[])
>    at org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:767)
>    at org.apache.cassandra.db.DataTracker.replaceReaders(DataTracker.java:408)
>    at 
> org.apache.cassandra.db.DataTracker.replaceWithNewInstances(DataTracker.java:312)
>    at 
> org.apache.cassandra.io.sstable.SSTableRewriter.moveStarts(SSTableRewriter.java:341)
>    at 
> org.apache.cassandra.io.sstable.SSTableRewriter.abort(SSTableRewriter.java:202)
>    at org.apache.cassandra.db.compaction.Scrubber.scrub(Scrubber.java:277)
>    at 
> org.apache.cassandra.db.ScrubTest.testScrubCorruptedCounterRow(ScrubTest.java:152)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to