[
https://issues.apache.org/jira/browse/CASSANDRA-7334?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14016091#comment-14016091
]
Ala' Alkhaldi commented on CASSANDRA-7334:
------------------------------------------
The test loops infinitely inside CommitLogSegmentManager.resetUnsafe() because
segmentManagementTasks stays nonempty.
Explanation:
# _resetUnsafe()_ clears the list of active and available sigments without
closing thier mmaped buffers.
# Then, when the CommitLog.recover() is called, it tries to recycle the
segments which leads to FSWriteError inside CommitLogSegment().
# This error is not caught in CommitLogSegmentManager() , where the tasks are
executed, which leads to existing the run loop for the manager thread. This
loop is responsible of consuming the tasks in segmentManagementTasks
# segmentManagementTasks stays nonempty and
CommitLogSegmentManager.resetUnsafe() loops infinitely when it is called in the
next time.
Solution:
Close the segments' buffers during resetUnsafe(). The batch is attached.
Notes:
# resetUnsafe() is used by multiple tests. all of them are tested to be run
successfully on Windows ( I will test them on linux later today ).
# resetUnsafe() is supposed to be used only by testing code. However, it is
used in SchemaLoader.cleanupAndLeaveDirs()
# FSWriteError thrown by CommitLogSegment() is not caught in
CommitLogSegmentManager() and many other places which led not to detect the
failure of this issue. Is this the intended behavior?
> RecoveryManagerTest times out on Windows
> ----------------------------------------
>
> Key: CASSANDRA-7334
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7334
> Project: Cassandra
> Issue Type: Sub-task
> Reporter: Joshua McKenzie
> Assignee: Ala' Alkhaldi
> Priority: Minor
> Labels: Windows
> Fix For: 3.0
>
>
> Takes 11 seconds on a slow linux vm to pass, times out on Windows.
> {code:title=timeout message}
> [junit] Testcase:
> org.apache.cassandra.db.RecoveryManagerTest:testRecoverCounter: Caused an
> ERROR
> [junit] Timeout occurred. Please note the time in the report does not
> reflect the time until the timeout.
> [junit] junit.framework.AssertionFailedError: Timeout occurred. Please
> note the time in the report does not reflect the time until the timeout.
> [junit]
> [junit]
> [junit] Test org.apache.cassandra.db.RecoveryManagerTest FAILED (timeout)
> {code}
--
This message was sent by Atlassian JIRA
(v6.2#6252)