[
https://issues.apache.org/jira/browse/CASSANDRA-6557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13874625#comment-13874625
]
Benedict commented on CASSANDRA-6557:
-------------------------------------
I agree it doesn't need to be super-optimised, and we could use the design you
suggest, _if_ we were to modify CLS so that we can create it in the aAF method
with the correct segment id. As it stands, the segment ids are assigned in the
CLSM thread, so they need to be added to active/allocatingFrom in the order
they are added to available. I don't think this is a very neat solution,
though, as the CLSM thread needs to at least assign the first segment id, and
we'd need an intermediate object.
The alternative, of course, is to lock when we want to aAF. As you say, it's a
rare operation, and it would be fine to lock for it. I thought it was a bit
ugly in the original MT CL patch, but it doesn't absolutely *need* NBQ, so if
we want to avoid adding it for now, we can.
I had planned to move active and available to a single NBQ, with active a NBQV
on the available NBQ, at some point in the near future, as it more accurately
models what we're doing (it's just a single queue we have, in three parts, and
we just treat the parts slightly differently), but that is also not strictly
necessary. I think it simplifies the algorithm, though.
> CommitLogSegment may be duplicated in unlikely race scenario
> ------------------------------------------------------------
>
> Key: CASSANDRA-6557
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6557
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Environment: 2.1
> Reporter: Benedict
> Fix For: 2.1
>
>
> In the unlikely event that the thread that switched to a new CLS has not
> finished executing the cleanup of its switch by the time the CLS has finished
> being used, it is possible for the same segment to be 'switched' in again.
> This would be benign except that it is added to the activeSegments queue a
> second time also, which would permit it to be recycled twice, creating two
> different CLS objects in memory pointing to the same CLS on disk, after which
> all bets are off.
> The issue is highly unlikely to occur, but highly unlikely means it will
> probably happen eventually. I've fixed this based on my patch for
> CASSANDRA-5549, using the NonBlockingQueue I introduce there to simplify the
> logic and make it more obviously correct.
--
This message was sent by Atlassian JIRA
(v6.1.5#6160)