[
https://issues.apache.org/jira/browse/CASSANDRA-9104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Joshua McKenzie updated CASSANDRA-9104:
---------------------------------------
Attachment: 9104_RecoveryManager_v2.txt
9104_KeyCache_ScrubTest_v2.txt
h6. KeyCacheTest / ScrubTest
Revised. I agree that promoting the disabling of early re-open on Windows makes
it less brittle; I was focused on addressing the utest failure and recently
promoted disabling of early re-open on Windows in SSTRW already. Good call.
bq. Is this the right bail-out check? What happens if preemptive open is
enabled, but has a large interval that wasn't reached yet?
Not sure I follow - by checking for the interval being Long.MAX_VALUE, we're
checking to see whether or not early re-open is disabled. This change should
have no impact on cases where preemptive open is enabled with a high value as
that will still be less than Long.MAX_VALUE and should process the
abort.moveStarts call as before.
bq. Could we also add the Linux test you did, using
SSTableRewriter.overrideOpenInterval?
Added a noEarlyOpen test.
Attached KeyCache and Scrub as single patch as they're touching similar
components.
h6. RecoveryTest
On first run of the CLSM, regardless of the flag a new segment will be created:
{noformat}
if (availableSegments.isEmpty() && (activeSegments.isEmpty() ||
createReserveSegments))
{noformat}
If availableSegments and activeSegments are both empty (i.e. on startup or
after utest clear), a new segment will be created and potentially race with the
recover() process. This problem is arising in unit-tests specifically as
there's no buffer between starting the CLSM and recovering the files whereas in
CassandraDaemon.java, we go through quite a bit between static init/start of
CLSM and the recover call. I've added a new method to register w/the signal in
CLSM for available segments and added that to CommitLog.recover as this
resolves the unit test issues and gives us more robust protection against this
race in regular startup rather than relying on CassandraDaemon.setup taking
long enough to get the activeSegment into the CLQ.
Along with the above, I adjusted the various unsafe start/stop methods to both
avoid calling wakeManager and adjust the reserve flag as appropriate.
> 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.0
>
> 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)