[
https://issues.apache.org/jira/browse/CASSANDRA-9104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14492342#comment-14492342
]
Branimir Lambov commented on CASSANDRA-9104:
--------------------------------------------
{{KeyCacheTest 180}}: This is too brittle; the isWindows() check and choice in
the static initialization section in SSTableRewriter should be moved to
DatabaseDescriptor.applyConfig() so that getSSTablePreempiveOpenIntervalInMB()
returns the OS-adjusted value.
{{SSTableRewriter 283}}: Is this the right bail-out check? What happens if
preemptive open is enabled, but has a large interval that wasn't reached yet?
Could we also add the Linux test you did, using
SSTableRewriter.overrideOpenInterval?
{{CommitLogSegmentManager 471}}: Is this change necessary?
bq. The recover() path was sneaking in between creation of a CommitLogSegment
and addition of that segment to the CLQ inside CLSM, so when we got the list of
files in the folder and attempted to filter them to just unmanaged files, that
segment was considered unmanaged. We go through recovery, we try to delete the
file, Windows barfs since it's memory-mapped.
This sounds like a real bug, and it should show up (in another form) on Linux
as well. The file should not be deleted; otherwise subsequent writes to the
commit log will break. During normal commitlog startup the
CLSM.createReserveSegments flag is used specifically to avoid this. During
tests CL.resetUnsafe() does not clear it, and even calls CLSM.wakeManager()
which makes the problem much more likely. The proper fix is to clear the flag
at the start of CLSM.stopUnsafe() and only call
CLSM.enableReserveSegmentCreation() if the reset is not to be followed by
CL.recover() (which finishes with that call). In either case CLSM.wakeManager()
shouldn't be called directly.
> 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_RecoveryManager.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)