[ 
https://issues.apache.org/jira/browse/IGNITE-28613?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Alex Abashev updated IGNITE-28613:
----------------------------------
    Description: 
h2. Summary

{{GridNioServer.close(GridSelectorNioSessionImpl, IgniteCheckedException, 
boolean)}} releases the session's direct {{ByteBuffer}}s via 
{{GridUnsafe.cleanDirectBuffer(...)}} *before* it cancels the underlying 
{{SelectionKey}}. While that close runs, the corresponding 
{{AbstractNioClientWorker}} can still be inside {{processRead(...)}} on the 
same session — it reads from a buffer whose native memory was just freed and 
now contains the allocator's poison pattern ({{0xEFEFEFEF}} on macOS / glibc / 
common debug allocators). The buffered parser then sees a 4-byte "size" of 
{{-269488145}} and aborts the session with {{IgniteCheckedException: Invalid 
message size}}.

The exception is caught by the filter chain and logged at {{ERROR}} level; 
functionally the session is already closing, so no test assertion fires. But 
the noise pollutes logs, breaks any "no errors during teardown" CI guards, and 
the underlying use-after-free is a latent bug — on a different allocator the 
freed memory could be reused by another thread and produce silently corrupted 
reads.

h2. Symptom

Reproducible (intermittent) on {{GridNioSelfTest#testAsyncSendReceive}} and 
other tests that close NIO sessions with traffic still in flight. Stack trace 
from a failing run on the current master:

{noformat}
[ERROR][grid-nio-worker-0-#42%nio-test-grid%][] Closing NIO session because of 
unhandled exception.
class org.apache.ignite.IgniteCheckedException: Invalid message size: -269488145
    at 
org.apache.ignite.internal.util.nio.GridNioServerBuffer.read(GridNioServerBuffer.java:82)
    at 
org.apache.ignite.internal.util.nio.GridBufferedParser.decode(GridBufferedParser.java:69)
    at 
org.apache.ignite.internal.util.nio.GridBufferedParser.decode(GridBufferedParser.java:36)
    at 
org.apache.ignite.internal.util.nio.GridNioCodecFilter.onMessageReceived(GridNioCodecFilter.java:114)
    at 
org.apache.ignite.internal.util.nio.GridNioFilterAdapter.proceedMessageReceived(GridNioFilterAdapter.java:110)
    at 
org.apache.ignite.internal.util.nio.GridNioServer$HeadFilter.onMessageReceived(GridNioServer.java:3791)
    at 
org.apache.ignite.internal.util.nio.GridNioFilterChain.onMessageReceived(GridNioFilterChain.java:176)
    at 
org.apache.ignite.internal.util.nio.GridNioServer$ByteBufferNioClientWorker.processRead(GridNioServer.java:1235)
    at 
org.apache.ignite.internal.util.nio.GridNioServer$AbstractNioClientWorker.processSelectedKeysOptimized(GridNioServer.java:2564)
    at 
org.apache.ignite.internal.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2319)
    at 
org.apache.ignite.internal.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1950)
    at 
org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:125)
    at java.base/java.lang.Thread.run(Thread.java:829)
{noformat}

{{0xEFEFEFEF}} = {{-269488145}} interpreted as a signed int. Java's 
{{ByteBuffer.allocateDirect}} returns zero-initialized memory, so this pattern 
is *not* coming from the original allocation — it's the allocator's poison 
written when the off-heap region was freed.

h2. Root cause

{{GridNioServer.close(...)}} (around lines 2879-2893):

{code:java}
if (ses.setClosed()) {
    ses.onClosed();

    if (directBuf) {
        if (ses.writeBuffer() != null)
            GridUnsafe.cleanDirectBuffer(ses.writeBuffer());   // (1) free 
off-heap

        if (ses.readBuffer() != null)
            GridUnsafe.cleanDirectBuffer(ses.readBuffer());    // (1)
    }

    if (closeSock)
        closeKey(ses.key());                                    // (2) detach 
selector
    else
        ses.key().cancel();
    ...
}
{code}

The selection key is cancelled at step (2) — that's the only thing that 
prevents the worker from selecting this session again. Step (1) runs first, so 
there is a window where:

* {{readBuffer}} 's native memory is freed.
* {{key}} is still registered with the selector.
* {{AbstractNioClientWorker}} can still pick up a {{readyOps()}} on this key 
from the most recent {{select()}} and call {{processRead(ses)}} → 
{{filterChain.onMessageReceived(ses, buf)}} → parser reads the freed bytes.

It's effectively a use-after-free; the {{0xEFEFEFEF}} bytes are the symptom 
rather than the cause. With a more aggressive allocator the same window could 
surface as silent data corruption rather than a thrown exception, because some 
unrelated allocation could land at the same address before the worker reads it.

h2. Why now

The race has been present since {{cleanDirectBuffer}} was added to 
{{close(...)}}. It surfaces more often on:

* Apple Silicon / recent macOS — the system allocator hands freed pages back to 
the kernel faster, so the poison pattern lands almost immediately.
* JDK 17+ direct-buffer cleaner — runs synchronously on {{cleanDirectBuffer}} 
call rather than on GC, narrowing the window for "happens to still hold zeros" 
luck.

It is *not* a regression from any recent message-protocol or serializer 
refactor (IGNITE-26508, IGNITE-28221, etc.). The same code path predates them.

h2. Reproduction

* Run {{GridNioSelfTest#testAsyncSendReceive}} on macOS arm64 with a recent JDK 
11+ build. The JUnit assertion still passes (the latch fires before close), but 
the surefire log contains the stack above on a meaningful fraction of runs.
* Synthetic stress: send N messages, immediately {{ses.close()}} from the test 
thread while the worker is still draining the receive buffer. {{N=10}} is 
enough to hit the race a few times in a hundred.

h2. Proposed fix

Cancel/close the selection key *before* freeing the direct buffers, so the 
worker is guaranteed off the key by the time the off-heap memory is released. 
Diff:

{code:java}
// GridNioServer.java, inside close(...) after ses.onClosed():
if (closeSock)
    closeKey(ses.key());
else
    ses.key().cancel();

if (directBuf) {
    if (ses.writeBuffer() != null)
        GridUnsafe.cleanDirectBuffer(ses.writeBuffer());

    if (ses.readBuffer() != null)
        GridUnsafe.cleanDirectBuffer(ses.readBuffer());
}
{code}

Note: {{SelectionKey.cancel()}} only marks the key for cancellation; it is 
finalized on the next {{select()}}. The worker can still be inside an 
in-progress {{processRead}} when we reach the buffer cleanup. To make the fix 
airtight we additionally need a happens-before barrier between "no more reads 
on this session" and "free the buffer".

Two options for the barrier:

# *Defer the cleanup to GC.* Drop {{cleanDirectBuffer}} from {{close(...)}} and 
let the {{Cleaner}} attached to the {{ByteBuffer}} (JDK 9+) reclaim the 
off-heap memory when the buffer is unreachable. After 
{{ses.removeMeta(BUF_META_KEY)}} and the session reference is dropped, the 
buffer becomes garbage like any other. Cost: a slightly later release of native 
memory; benefit: no use-after-free is even possible.
# *Synchronize with the owning worker.* Have {{close(...)}} post a 
"drain-and-release" task to the worker that owns the session and let the worker 
do the {{cleanDirectBuffer}} after its current {{processRead}} finishes. 
Heavier, but keeps the eager-release behaviour for users who care about it.

Option 1 is recommended — {{cleanDirectBuffer}} is an optimisation, not a 
correctness requirement, and removing the eager free is a one-liner. Acceptance 
test below confirms there is no measurable native-memory growth in steady state 
because sessions are short-lived and the {{Cleaner}} drains them at GC pace.

h2. Acceptance criteria

* {{GridNioSelfTest#testAsyncSendReceive}} produces no {{Invalid message size}} 
entries in the surefire log over 50 consecutive runs (currently 5-15% of runs 
hit it).
* No {{IgniteCheckedException}} originating from {{GridNioServerBuffer.read}} 
appears in any nio test class on JDK 11/17/21 on Linux x86\_64 and macOS 
aarch64.
* JFR / native-memory tracking shows direct-buffer reclaim within one GC cycle 
of session close (option 1) — no steady-state growth on a stress test that 
opens/closes 100k sessions.
* {{GridCommunicationSelfTest}} and {{IgniteCacheMessageRecoveryAbstractTest}} 
suites stay green — they exercise the same close path under load.

h2. Out of scope

* The buffered parser's protocol itself is fine; no need to harden the size 
validation. The garbage size value is a symptom, not the bug.
* No changes to {{DirectMessageWriter}} / {{DirectMessageReader}} or to the 
message serializer contract — those are separate concerns from session 
lifecycle.

> Race in GridNioServer.close: direct buffers are freed before the NIO worker 
> stops reading them
> ----------------------------------------------------------------------------------------------
>
>                 Key: IGNITE-28613
>                 URL: https://issues.apache.org/jira/browse/IGNITE-28613
>             Project: Ignite
>          Issue Type: Task
>            Reporter: Alex Abashev
>            Assignee: Anton Vinogradov
>            Priority: Minor
>              Labels: IEP-132, ise
>
> h2. Summary
> {{GridNioServer.close(GridSelectorNioSessionImpl, IgniteCheckedException, 
> boolean)}} releases the session's direct {{ByteBuffer}}s via 
> {{GridUnsafe.cleanDirectBuffer(...)}} *before* it cancels the underlying 
> {{SelectionKey}}. While that close runs, the corresponding 
> {{AbstractNioClientWorker}} can still be inside {{processRead(...)}} on the 
> same session — it reads from a buffer whose native memory was just freed and 
> now contains the allocator's poison pattern ({{0xEFEFEFEF}} on macOS / glibc 
> / common debug allocators). The buffered parser then sees a 4-byte "size" of 
> {{-269488145}} and aborts the session with {{IgniteCheckedException: Invalid 
> message size}}.
> The exception is caught by the filter chain and logged at {{ERROR}} level; 
> functionally the session is already closing, so no test assertion fires. But 
> the noise pollutes logs, breaks any "no errors during teardown" CI guards, 
> and the underlying use-after-free is a latent bug — on a different allocator 
> the freed memory could be reused by another thread and produce silently 
> corrupted reads.
> h2. Symptom
> Reproducible (intermittent) on {{GridNioSelfTest#testAsyncSendReceive}} and 
> other tests that close NIO sessions with traffic still in flight. Stack trace 
> from a failing run on the current master:
> {noformat}
> [ERROR][grid-nio-worker-0-#42%nio-test-grid%][] Closing NIO session because 
> of unhandled exception.
> class org.apache.ignite.IgniteCheckedException: Invalid message size: 
> -269488145
>     at 
> org.apache.ignite.internal.util.nio.GridNioServerBuffer.read(GridNioServerBuffer.java:82)
>     at 
> org.apache.ignite.internal.util.nio.GridBufferedParser.decode(GridBufferedParser.java:69)
>     at 
> org.apache.ignite.internal.util.nio.GridBufferedParser.decode(GridBufferedParser.java:36)
>     at 
> org.apache.ignite.internal.util.nio.GridNioCodecFilter.onMessageReceived(GridNioCodecFilter.java:114)
>     at 
> org.apache.ignite.internal.util.nio.GridNioFilterAdapter.proceedMessageReceived(GridNioFilterAdapter.java:110)
>     at 
> org.apache.ignite.internal.util.nio.GridNioServer$HeadFilter.onMessageReceived(GridNioServer.java:3791)
>     at 
> org.apache.ignite.internal.util.nio.GridNioFilterChain.onMessageReceived(GridNioFilterChain.java:176)
>     at 
> org.apache.ignite.internal.util.nio.GridNioServer$ByteBufferNioClientWorker.processRead(GridNioServer.java:1235)
>     at 
> org.apache.ignite.internal.util.nio.GridNioServer$AbstractNioClientWorker.processSelectedKeysOptimized(GridNioServer.java:2564)
>     at 
> org.apache.ignite.internal.util.nio.GridNioServer$AbstractNioClientWorker.bodyInternal(GridNioServer.java:2319)
>     at 
> org.apache.ignite.internal.util.nio.GridNioServer$AbstractNioClientWorker.body(GridNioServer.java:1950)
>     at 
> org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:125)
>     at java.base/java.lang.Thread.run(Thread.java:829)
> {noformat}
> {{0xEFEFEFEF}} = {{-269488145}} interpreted as a signed int. Java's 
> {{ByteBuffer.allocateDirect}} returns zero-initialized memory, so this 
> pattern is *not* coming from the original allocation — it's the allocator's 
> poison written when the off-heap region was freed.
> h2. Root cause
> {{GridNioServer.close(...)}} (around lines 2879-2893):
> {code:java}
> if (ses.setClosed()) {
>     ses.onClosed();
>     if (directBuf) {
>         if (ses.writeBuffer() != null)
>             GridUnsafe.cleanDirectBuffer(ses.writeBuffer());   // (1) free 
> off-heap
>         if (ses.readBuffer() != null)
>             GridUnsafe.cleanDirectBuffer(ses.readBuffer());    // (1)
>     }
>     if (closeSock)
>         closeKey(ses.key());                                    // (2) detach 
> selector
>     else
>         ses.key().cancel();
>     ...
> }
> {code}
> The selection key is cancelled at step (2) — that's the only thing that 
> prevents the worker from selecting this session again. Step (1) runs first, 
> so there is a window where:
> * {{readBuffer}} 's native memory is freed.
> * {{key}} is still registered with the selector.
> * {{AbstractNioClientWorker}} can still pick up a {{readyOps()}} on this key 
> from the most recent {{select()}} and call {{processRead(ses)}} → 
> {{filterChain.onMessageReceived(ses, buf)}} → parser reads the freed bytes.
> It's effectively a use-after-free; the {{0xEFEFEFEF}} bytes are the symptom 
> rather than the cause. With a more aggressive allocator the same window could 
> surface as silent data corruption rather than a thrown exception, because 
> some unrelated allocation could land at the same address before the worker 
> reads it.
> h2. Why now
> The race has been present since {{cleanDirectBuffer}} was added to 
> {{close(...)}}. It surfaces more often on:
> * Apple Silicon / recent macOS — the system allocator hands freed pages back 
> to the kernel faster, so the poison pattern lands almost immediately.
> * JDK 17+ direct-buffer cleaner — runs synchronously on {{cleanDirectBuffer}} 
> call rather than on GC, narrowing the window for "happens to still hold 
> zeros" luck.
> It is *not* a regression from any recent message-protocol or serializer 
> refactor (IGNITE-26508, IGNITE-28221, etc.). The same code path predates them.
> h2. Reproduction
> * Run {{GridNioSelfTest#testAsyncSendReceive}} on macOS arm64 with a recent 
> JDK 11+ build. The JUnit assertion still passes (the latch fires before 
> close), but the surefire log contains the stack above on a meaningful 
> fraction of runs.
> * Synthetic stress: send N messages, immediately {{ses.close()}} from the 
> test thread while the worker is still draining the receive buffer. {{N=10}} 
> is enough to hit the race a few times in a hundred.
> h2. Proposed fix
> Cancel/close the selection key *before* freeing the direct buffers, so the 
> worker is guaranteed off the key by the time the off-heap memory is released. 
> Diff:
> {code:java}
> // GridNioServer.java, inside close(...) after ses.onClosed():
> if (closeSock)
>     closeKey(ses.key());
> else
>     ses.key().cancel();
> if (directBuf) {
>     if (ses.writeBuffer() != null)
>         GridUnsafe.cleanDirectBuffer(ses.writeBuffer());
>     if (ses.readBuffer() != null)
>         GridUnsafe.cleanDirectBuffer(ses.readBuffer());
> }
> {code}
> Note: {{SelectionKey.cancel()}} only marks the key for cancellation; it is 
> finalized on the next {{select()}}. The worker can still be inside an 
> in-progress {{processRead}} when we reach the buffer cleanup. To make the fix 
> airtight we additionally need a happens-before barrier between "no more reads 
> on this session" and "free the buffer".
> Two options for the barrier:
> # *Defer the cleanup to GC.* Drop {{cleanDirectBuffer}} from {{close(...)}} 
> and let the {{Cleaner}} attached to the {{ByteBuffer}} (JDK 9+) reclaim the 
> off-heap memory when the buffer is unreachable. After 
> {{ses.removeMeta(BUF_META_KEY)}} and the session reference is dropped, the 
> buffer becomes garbage like any other. Cost: a slightly later release of 
> native memory; benefit: no use-after-free is even possible.
> # *Synchronize with the owning worker.* Have {{close(...)}} post a 
> "drain-and-release" task to the worker that owns the session and let the 
> worker do the {{cleanDirectBuffer}} after its current {{processRead}} 
> finishes. Heavier, but keeps the eager-release behaviour for users who care 
> about it.
> Option 1 is recommended — {{cleanDirectBuffer}} is an optimisation, not a 
> correctness requirement, and removing the eager free is a one-liner. 
> Acceptance test below confirms there is no measurable native-memory growth in 
> steady state because sessions are short-lived and the {{Cleaner}} drains them 
> at GC pace.
> h2. Acceptance criteria
> * {{GridNioSelfTest#testAsyncSendReceive}} produces no {{Invalid message 
> size}} entries in the surefire log over 50 consecutive runs (currently 5-15% 
> of runs hit it).
> * No {{IgniteCheckedException}} originating from {{GridNioServerBuffer.read}} 
> appears in any nio test class on JDK 11/17/21 on Linux x86\_64 and macOS 
> aarch64.
> * JFR / native-memory tracking shows direct-buffer reclaim within one GC 
> cycle of session close (option 1) — no steady-state growth on a stress test 
> that opens/closes 100k sessions.
> * {{GridCommunicationSelfTest}} and 
> {{IgniteCacheMessageRecoveryAbstractTest}} suites stay green — they exercise 
> the same close path under load.
> h2. Out of scope
> * The buffered parser's protocol itself is fine; no need to harden the size 
> validation. The garbage size value is a symptom, not the bug.
> * No changes to {{DirectMessageWriter}} / {{DirectMessageReader}} or to the 
> message serializer contract — those are separate concerns from session 
> lifecycle.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to