magibney commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1684426889
##########
lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java:
##########
@@ -157,6 +157,8 @@ static Arena getSharedArena(
arenas.computeIfAbsent(key, s -> new RefCountedSharedArena(s, () ->
arenas.remove(s)));
if (refCountedArena.acquire()) {
return refCountedArena;
+ } else {
+ arenas.remove(key);
Review Comment:
I think there's a race condition here -- if 2 threads both get a
`refCountedArena` that's at the limit threshold and both fail to acquire, then
one might remove an Arena just created by the other. I think this could be
addressed by turning the removal fallback into a conditionally-computed
replacement while holding the lock on the key, i.e. instead of
`arenas.remove(key)`:
```java
var refCountedArena =
arenas.computeIfAbsent(key, s -> new RefCountedSharedArena(s, () ->
arenas.remove(s)));
if (refCountedArena.acquire()) {
return refCountedArena;
} else {
return arenas.compute(key, (s, v) -> {
if (v != null && v.acquire()) {
return v;
} else {
v = new RefCountedSharedArena(s, () -> arenas.remove(s));
v.acquire(); // guaranteed to succeed
return v;
}
});
}
```
Then we don't need a loop, and this covers both the remove-on-release and
remove-on-acquire cases.
##########
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##########
@@ -47,9 +48,12 @@ boolean acquire() {
int value;
while (true) {
value = state.get();
- if (value < OPEN) {
+ if (value >= LIMIT) {
Review Comment:
Hmm, but I think that unless we _do_ separate "total lifetime count" from
state, then this is unlikely to ever hit in practice. The situation we're
trying to protect against is repeated map and un-map, which, for a single
segment or segment/gen, could easily stay below an outstanding limit of 256,
while allocating an unlimited amount of VM space. (Accordingly, assuming we go
with separate "total lifetime count" here, a higher limit would probably be
appropriate).
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]