-
Notifications
You must be signed in to change notification settings - Fork 24.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Delegate Ref Counting to ByteBuf in Netty Transport #81096
Delegate Ref Counting to ByteBuf in Netty Transport #81096
Conversation
Tracking down recent memory leaks was made unnecessarily hard by wrapping the `ByteBuf` ref couting with our own counter. This way, we would not record the increments and decrements on the Netty leak tracker, making it useless as far as identifying the concrete source of a request with the logged leak only containing touch points up until our inbound handler code.
Pinging @elastic/es-distributed (Team:Distributed) |
try { | ||
buffer.retain(); | ||
} catch (RuntimeException e) { | ||
assert refCount() == 0 : "should only fail if fully released but ref count was [" + refCount() + "]"; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not the most elegant solution, but Netty ref counting simply doesn't have this functionality. Also, doesn't really matter as in practice the tryIncRef
call seems not be used on the network buffers so there's performance relevant impact here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Perhaps check if buffer.refCnt() == 0
first? We still need to catch an exception here but it'd be much rarer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
++ added, makes sense
public boolean hasReferences() { | ||
return true; | ||
public int refCount() { | ||
return 1; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The point of hasReferences()
is to avoid having to return a fake value like this for implementations that always leak. There's something weird about having a refcount which incRef()
and decRef()
don't affect.
Do we need to expose the exact refcount here? If not, can we put this back to how it was?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
is to avoid having to return a fake value like this for implementations that always leak
Right. Unfortunately, this would mean removing org.elasticsearch.common.bytes.ReleasableBytesReference#refCount
which is quite the large change (although we eventually only make use of the number in tests seemingly) and doesn't really seem worth the effort to avoid a cosmetic issue like this?
I guess Netty does the same and uses fake 1
for the ref count in EmptyByteBuf
and the like so I figured it's good enough for us here as well :)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah I see. Still, half of those usages are assert refCount() > 0;
which just become assert hasReferences();
and the others in tests are pretty much all equally trivial. I almost made that change myself when introducing hasReferences()
. I'd rather do the Right Thing here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm ok, let me try
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok I concede that this was much easier than expected. 3cf250c ... effectively only a single spot where we lost some trivial coverage (ref count == 2 assertion)
try { | ||
buffer.retain(); | ||
} catch (RuntimeException e) { | ||
assert refCount() == 0 : "should only fail if fully released but ref count was [" + refCount() + "]"; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Perhaps check if buffer.refCnt() == 0
first? We still need to catch an exception here but it'd be much rarer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
@@ -107,8 +107,6 @@ public void testDecode() throws IOException { | |||
final Object endMarker = fragments.get(1); | |||
|
|||
assertEquals(messageBytes, content); | |||
// Ref count is incremented since the bytes are forwarded as a fragment | |||
assertEquals(2, releasable2.refCount()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You could reasonably keep this coverage by releasing releasable2
and asserting that it still hasReferences()
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
++ brought this back
Thanks David! |
💔 Backport failed
You can use sqren/backport to manually backport by running |
* upstream/master: (150 commits) Fix ComposableIndexTemplate equals when composed_of is null (elastic#80864) Optimize DLS bitset building for matchAll query (elastic#81030) URL option for BaseRunAsSuperuserCommand (elastic#81025) Less Verbose Serialization of Snapshot Failure in SLM Metadata (elastic#80942) Fix shadowed vars pt7 (elastic#80996) Fail shards early when we can detect a type missmatch (elastic#79869) Delegate Ref Counting to ByteBuf in Netty Transport (elastic#81096) Clarify `unassigned.reason` docs (elastic#81017) Strip blocks from settings for reindex targets (elastic#80887) Split off the values supplier for ScriptDocValues (elastic#80635) [ML] Switch message and detail for model snapshot deprecations (elastic#81108) [DOCS] Update xrefs for snapshot restore docs (elastic#81023) [ML] Updates visiblity of validate API (elastic#81061) Track histogram of transport handling times (elastic#80581) [ML] Fix datafeed preview with remote indices (elastic#81099) [ML] Fix acceptable model snapshot versions in ML deprecation checker (elastic#81060) [ML] Add logging for failing PyTorch test (elastic#81044) Extending the timeout waiting for snapshot to be ready (elastic#81018) [ML] Fix incorrect logging of unexpected model size error (elastic#81089) [ML] Make inference timeout test more reliable (elastic#81094) ... # Conflicts: # server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java
* upstream/master: (55 commits) Fix ComposableIndexTemplate equals when composed_of is null (elastic#80864) Optimize DLS bitset building for matchAll query (elastic#81030) URL option for BaseRunAsSuperuserCommand (elastic#81025) Less Verbose Serialization of Snapshot Failure in SLM Metadata (elastic#80942) Fix shadowed vars pt7 (elastic#80996) Fail shards early when we can detect a type missmatch (elastic#79869) Delegate Ref Counting to ByteBuf in Netty Transport (elastic#81096) Clarify `unassigned.reason` docs (elastic#81017) Strip blocks from settings for reindex targets (elastic#80887) Split off the values supplier for ScriptDocValues (elastic#80635) [ML] Switch message and detail for model snapshot deprecations (elastic#81108) [DOCS] Update xrefs for snapshot restore docs (elastic#81023) [ML] Updates visiblity of validate API (elastic#81061) Track histogram of transport handling times (elastic#80581) [ML] Fix datafeed preview with remote indices (elastic#81099) [ML] Fix acceptable model snapshot versions in ML deprecation checker (elastic#81060) [ML] Add logging for failing PyTorch test (elastic#81044) Extending the timeout waiting for snapshot to be ready (elastic#81018) [ML] Fix incorrect logging of unexpected model size error (elastic#81089) [ML] Make inference timeout test more reliable (elastic#81094) ...
* upstream/master: (55 commits) Fix ComposableIndexTemplate equals when composed_of is null (elastic#80864) Optimize DLS bitset building for matchAll query (elastic#81030) URL option for BaseRunAsSuperuserCommand (elastic#81025) Less Verbose Serialization of Snapshot Failure in SLM Metadata (elastic#80942) Fix shadowed vars pt7 (elastic#80996) Fail shards early when we can detect a type missmatch (elastic#79869) Delegate Ref Counting to ByteBuf in Netty Transport (elastic#81096) Clarify `unassigned.reason` docs (elastic#81017) Strip blocks from settings for reindex targets (elastic#80887) Split off the values supplier for ScriptDocValues (elastic#80635) [ML] Switch message and detail for model snapshot deprecations (elastic#81108) [DOCS] Update xrefs for snapshot restore docs (elastic#81023) [ML] Updates visiblity of validate API (elastic#81061) Track histogram of transport handling times (elastic#80581) [ML] Fix datafeed preview with remote indices (elastic#81099) [ML] Fix acceptable model snapshot versions in ML deprecation checker (elastic#81060) [ML] Add logging for failing PyTorch test (elastic#81044) Extending the timeout waiting for snapshot to be ready (elastic#81018) [ML] Fix incorrect logging of unexpected model size error (elastic#81089) [ML] Make inference timeout test more reliable (elastic#81094) ...
Tracking down recent memory leaks was made unnecessarily hard
by wrapping the
ByteBuf
ref counting with our own counter. Thisway, we would not record the increments and decrements on the Netty
leak tracker, making it useless as far as identifying the concrete
source of a request with the logged leak only containing touch points
up until our inbound handler code.
As a side note: It would also be nice to do the same on the rest layer, but it's quite a bit harder there since we don't really manage a ref count for rest content today, instead we just delay releasing content until we send the response for some messages.