[
https://issues.apache.org/jira/browse/IGNITE-28606?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alex Abashev updated IGNITE-28606:
----------------------------------
Description:
h2. Summary
{{KeyCacheObjectImpl.prepareMarshal(CacheObjectValueContext)}} and
{{CacheObjectImpl.prepareMarshal(...)}} rely on an unsynchronized {{if
(valBytes == null)}} guard as their idempotency check. Under concurrent access
— e.g. when the same {{CacheObject}} instance is referenced from multiple
transaction entries that are marshalled from different threads — two callers
can both observe {{valBytes == null}}, both invoke {{ctx.marshal(val)}}, and
race to publish their result. The work is duplicated (CPU + allocations) and
the final field value depends on write ordering.
h2. Where observed
Surfaced during IGNITE-28520 Phase 2 work (migration of hand-written
{{prepareMarshal}} to auto-invocation of generated
{{MessageSerializer.prepareMarshalCacheObjects}}). An experimental "called at
most once per CO instance" assertion was added to {{CacheObjectAdapter}} to
validate the new path. Running {{IgniteCacheQueryMultiThreadedSelfTest}} with
{{-ea}} produced:
{noformat}
java.lang.AssertionError: prepareMarshal called more than once on
KeyCacheObjectImpl
at
org.apache.ignite.internal.processors.cache.CacheObjectAdapter.assertFirstPrepareMarshal(CacheObjectAdapter.java:53)
at
org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl.prepareMarshal(KeyCacheObjectImpl.java:119)
at
org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.marshal(IgniteTxEntry.java:946)
at
org.apache.ignite.internal.processors.cache.GridCacheMessage.marshalTx(GridCacheMessage.java:380)
at
org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest.prepareMarshal(GridDistributedTxPrepareRequest.java:379)
at
org.apache.ignite.internal.processors.cache.GridCacheIoManager.onSend(GridCacheIoManager.java:1152)
at
org.apache.ignite.internal.processors.cache.GridCacheIoManager.send(GridCacheIoManager.java:1223)
...
{noformat}
The assertion was rolled back because it was incompatible with today's
intentionally-racy design (see IGNITE-28520 Phase 2 audit, "Rejected idea:
per-CO marshalled-once assertion"). The underlying race remains and is the
subject of this ticket.
h2. Root cause
{code:java}
// KeyCacheObjectImpl
@Override public void prepareMarshal(CacheObjectValueContext ctx) throws
IgniteCheckedException {
if (valBytes == null) // (1) unsynchronized read
valBytes = ctx.marshal(val); // (2) unsynchronized write
}
{code}
Two threads executing {{(1)}} concurrently both see {{null}}, both execute
{{(2)}}. {{valBytes}} is not {{volatile}}, so publication is also unspecified —
another thread reading the field after the race can observe {{null}} even after
both writers completed on other cores.
Same pattern in {{CacheObjectImpl.prepareMarshal}}.
{{CacheObjectByteArrayImpl.prepareMarshal}} is a no-op and is unaffected.
h2. Why it's a functional no-op today (but still a bug)
* {{val}} is effectively immutable once the {{CacheObject}} is constructed.
* {{ctx.marshal(val)}} is deterministic: equal inputs produce equal byte arrays.
* Final read by the NIO writer goes through {{writeExternal}} / {{putValue}} /
{{valueBytesLength}}, each of which calls {{valueBytes(ctx)}} that re-does the
idempotency check and will populate {{valBytes}} on the calling thread if both
racers somehow lost their writes.
So the race does not corrupt on-wire output in practice. What it does cost:
* *Wasted CPU*: duplicate {{Marshaller.marshal}} calls on contended keys (hot
paths during tx prepare when the same key appears in multiple {{IgniteTxEntry}}
instances).
* *Wasted allocation*: each losing racer produces a {{byte[]}} that's
immediately garbage.
* *Debugging friction*: any future audit or assertion that presumes "marshal
happens once per CO" cannot be added without first fixing this.
* *Specification drift*: the contract is effectively "idempotent under
concurrency by accident" — reliant on value immutability and marshaller
determinism rather than stated.
h2. Reproduction
* Run {{IgniteCacheQueryMultiThreadedSelfTest}} with {{-ea}} after temporarily
re-adding the assertion from the Phase 2 audit:
{code:java}
// in KeyCacheObjectImpl.prepareMarshal
if (valBytes == null) {
assert assertFirstPrepareMarshal();
valBytes = ctx.marshal(val);
}
{code}
Plus the transient flag + helper on {{CacheObjectAdapter}}.
* The assertion fires from the {{IgniteTxEntry.marshal → key.prepareMarshal}}
call site when a key is shared across concurrent prepare flows.
h2. Proposed fix
Synchronize the check-and-set. Two options, in order of preference:
# *Double-checked locking on a per-instance monitor*, keeping the hot no-op
branch lock-free:
{code:java}
@Override public void prepareMarshal(CacheObjectValueContext ctx) throws
IgniteCheckedException {
if (valBytes != null)
return;
synchronized (this) {
if (valBytes == null)
valBytes = ctx.marshal(val);
}
}
{code}
Requires making {{valBytes}} {{volatile}} for safe DCL. Uncontended cost: one
volatile read. Contended: {{synchronized}} on the {{CacheObject}} itself, which
is already the per-key identity.
# *Atomic publish via {{VarHandle}} / {{Unsafe.compareAndSetObject}}* on
{{valBytes}} — winner's bytes publish, loser's garbage. Avoids entering
{{synchronized}}, but requires a VarHandle field and the losing thread still
does the marshal work.
Option 1 eliminates both races and duplicate work; option 2 eliminates the race
but preserves duplicate work. Option 1 is recommended.
Mirror the same treatment in {{CacheObjectImpl.prepareMarshal}} (which calls
{{valueBytesFromValue(ctx)}} instead of {{ctx.marshal(val)}}).
h2. Scope
* {{modules/core}}:
** {{KeyCacheObjectImpl.prepareMarshal}}
** {{CacheObjectImpl.prepareMarshal}}
** {{CacheObjectAdapter.valBytes}} → {{volatile byte[]}}
* No wire-protocol change (field layout unchanged).
* No API change.
h2. Acceptance criteria
* Under {{-ea}}, the Phase 2 audit assertion re-added to
{{KeyCacheObjectImpl.prepareMarshal}} / {{CacheObjectImpl.prepareMarshal}} does
not fire in {{IgniteCacheQueryMultiThreadedSelfTest}} or other multi-threaded
suites.
* No measurable throughput regression on {{GridCachePutAllFailoverSelfTest}}
and {{GridCacheRebalancingSyncSelfTest}} (these exercise the same CO marshal
paths on the hot path).
* Unit test: two threads race on a fresh {{KeyCacheObjectImpl.prepareMarshal}}
— {{ctx.marshal(val)}} is invoked exactly once (verified via a counting
{{CacheObjectValueContext}} mock).
was:
## Summary
`KeyCacheObjectImpl.prepareMarshal(CacheObjectValueContext)` and
`CacheObjectImpl.prepareMarshal(...)` rely on an unsynchronized `if (valBytes
== null)` guard as their idempotency check. Under concurrent access — e.g. when
the same `CacheObject` instance is referenced from multiple transaction entries
that are marshalled from different threads — two callers can both observe
`valBytes == null`, both invoke `ctx.marshal(val)`, and race to publish their
result. The work is duplicated (CPU + allocations) and the final field value
depends on write ordering.
## Where observed
Surfaced during IGNITE-28520 Phase 2 work (migration of hand-written
`prepareMarshal` to auto-invocation of generated
`MessageSerializer.prepareMarshalCacheObjects`). An experimental "called at
most once per CO instance" assertion was added to `CacheObjectAdapter` to
validate the new path. Running `IgniteCacheQueryMultiThreadedSelfTest` with
`-ea` produced:
```
java.lang.AssertionError: prepareMarshal called more than once on
KeyCacheObjectImpl
at
org.apache.ignite.internal.processors.cache.CacheObjectAdapter.assertFirstPrepareMarshal(CacheObjectAdapter.java:53)
at
org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl.prepareMarshal(KeyCacheObjectImpl.java:119)
at
org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.marshal(IgniteTxEntry.java:946)
at
org.apache.ignite.internal.processors.cache.GridCacheMessage.marshalTx(GridCacheMessage.java:380)
at
org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest.prepareMarshal(GridDistributedTxPrepareRequest.java:379)
at
org.apache.ignite.internal.processors.cache.GridCacheIoManager.onSend(GridCacheIoManager.java:1152)
at
org.apache.ignite.internal.processors.cache.GridCacheIoManager.send(GridCacheIoManager.java:1223)
...
```
The assertion was rolled back because it was incompatible with today's
intentionally-racy design (see IGNITE-28520 Phase 2 audit, "Rejected idea:
per-CO marshalled-once assertion"). The underlying race remains and is the
subject of this ticket.
## Root cause
```java
// KeyCacheObjectImpl
@Override public void prepareMarshal(CacheObjectValueContext ctx) throws
IgniteCheckedException {
if (valBytes == null) // (1) unsynchronized read
valBytes = ctx.marshal(val); // (2) unsynchronized write
}
```
Two threads executing `(1)` concurrently both see `null`, both execute `(2)`.
`valBytes` is not `volatile`, so publication is also unspecified — another
thread reading the field after the race can observe `null` even after both
writers completed on other cores.
Same pattern in `CacheObjectImpl.prepareMarshal`.
`CacheObjectByteArrayImpl.prepareMarshal` is a no-op and is unaffected.
## Why it's a functional no-op today (but still a bug)
- `val` is effectively immutable once the `CacheObject` is constructed.
- `ctx.marshal(val)` is deterministic: equal inputs produce equal byte arrays.
- Final read by the NIO writer goes through `writeExternal` / `putValue` /
`valueBytesLength`, each of which calls `valueBytes(ctx)` that re-does the
idempotency check and will populate `valBytes` on the calling thread if both
racers somehow lost their writes.
So the race does not corrupt on-wire output in practice. What it does cost:
- **Wasted CPU**: duplicate `Marshaller.marshal` calls on contended keys (hot
paths during tx prepare when the same key appears in multiple `IgniteTxEntry`
instances).
- **Wasted allocation**: each losing racer produces a `byte[]` that's
immediately garbage.
- **Debugging friction**: any future audit or assertion that presumes "marshal
happens once per CO" cannot be added without first fixing this.
- **Specification drift**: the contract is effectively "idempotent under
concurrency by accident" — reliant on value immutability and marshaller
determinism rather than stated.
## Reproduction
- Run `IgniteCacheQueryMultiThreadedSelfTest` with `-ea` after temporarily
re-adding the assertion from the Phase 2 audit:
```java
// in KeyCacheObjectImpl.prepareMarshal
if (valBytes == null) {
assert assertFirstPrepareMarshal();
valBytes = ctx.marshal(val);
}
```
Plus the transient flag + helper on `CacheObjectAdapter`.
- The assertion fires from the `IgniteTxEntry.marshal → key.prepareMarshal`
call site when a key is shared across concurrent prepare flows.
## Proposed fix
Synchronize the check-and-set. Two options, in order of preference:
1. **Double-checked locking on a per-instance monitor**, keeping the hot no-op
branch lock-free:
```java
@Override public void prepareMarshal(CacheObjectValueContext ctx) throws
IgniteCheckedException {
if (valBytes != null)
return;
synchronized (this) {
if (valBytes == null)
valBytes = ctx.marshal(val);
}
}
```
Requires making `valBytes` `volatile` for safe DCL. Uncontended cost: one
volatile read. Contended: `synchronized` on the `CacheObject` itself, which is
already the per-key identity.
2. **Atomic publish via `VarHandle` / `Unsafe.compareAndSetObject`** on
`valBytes` — winner's bytes publish, loser's garbage. Avoids entering
`synchronized`, but requires a VarHandle field and the losing thread still does
the marshal work.
Option 1 eliminates both races and duplicate work; option 2 eliminates the race
but preserves duplicate work. Option 1 is recommended.
Mirror the same treatment in `CacheObjectImpl.prepareMarshal` (which calls
`valueBytesFromValue(ctx)` instead of `ctx.marshal(val)`).
## Scope
- `modules/core`:
- `KeyCacheObjectImpl.prepareMarshal`
- `CacheObjectImpl.prepareMarshal`
- `CacheObjectAdapter.valBytes` → `volatile byte[]`
- No wire-protocol change (field layout unchanged).
- No API change.
## Acceptance criteria
- Under `-ea`, the Phase 2 audit assertion re-added to
`KeyCacheObjectImpl.prepareMarshal` / `CacheObjectImpl.prepareMarshal` does not
fire in `IgniteCacheQueryMultiThreadedSelfTest` or other multi-threaded suites.
- No measurable throughput regression on `GridCachePutAllFailoverSelfTest` and
`GridCacheRebalancingSyncSelfTest` (these exercise the same CO marshal paths on
the hot path).
- Unit test: two threads race on a fresh `KeyCacheObjectImpl.prepareMarshal` —
`ctx.marshal(val)` is invoked exactly once (verified via a counting
`CacheObjectValueContext` mock).
> СacheObject.prepareMarshal is not thread-safe — concurrent callers duplicate
> marshalling work
> ---------------------------------------------------------------------------------------------
>
> Key: IGNITE-28606
> URL: https://issues.apache.org/jira/browse/IGNITE-28606
> Project: Ignite
> Issue Type: Task
> Reporter: Alex Abashev
> Assignee: Alex Abashev
> Priority: Minor
> Labels: IEP-132, ise
>
> h2. Summary
> {{KeyCacheObjectImpl.prepareMarshal(CacheObjectValueContext)}} and
> {{CacheObjectImpl.prepareMarshal(...)}} rely on an unsynchronized {{if
> (valBytes == null)}} guard as their idempotency check. Under concurrent
> access — e.g. when the same {{CacheObject}} instance is referenced from
> multiple transaction entries that are marshalled from different threads — two
> callers can both observe {{valBytes == null}}, both invoke
> {{ctx.marshal(val)}}, and race to publish their result. The work is
> duplicated (CPU + allocations) and the final field value depends on write
> ordering.
> h2. Where observed
> Surfaced during IGNITE-28520 Phase 2 work (migration of hand-written
> {{prepareMarshal}} to auto-invocation of generated
> {{MessageSerializer.prepareMarshalCacheObjects}}). An experimental "called at
> most once per CO instance" assertion was added to {{CacheObjectAdapter}} to
> validate the new path. Running {{IgniteCacheQueryMultiThreadedSelfTest}} with
> {{-ea}} produced:
> {noformat}
> java.lang.AssertionError: prepareMarshal called more than once on
> KeyCacheObjectImpl
> at
> org.apache.ignite.internal.processors.cache.CacheObjectAdapter.assertFirstPrepareMarshal(CacheObjectAdapter.java:53)
> at
> org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl.prepareMarshal(KeyCacheObjectImpl.java:119)
> at
> org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.marshal(IgniteTxEntry.java:946)
> at
> org.apache.ignite.internal.processors.cache.GridCacheMessage.marshalTx(GridCacheMessage.java:380)
> at
> org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest.prepareMarshal(GridDistributedTxPrepareRequest.java:379)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.onSend(GridCacheIoManager.java:1152)
> at
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.send(GridCacheIoManager.java:1223)
> ...
> {noformat}
> The assertion was rolled back because it was incompatible with today's
> intentionally-racy design (see IGNITE-28520 Phase 2 audit, "Rejected idea:
> per-CO marshalled-once assertion"). The underlying race remains and is the
> subject of this ticket.
> h2. Root cause
> {code:java}
> // KeyCacheObjectImpl
> @Override public void prepareMarshal(CacheObjectValueContext ctx) throws
> IgniteCheckedException {
> if (valBytes == null) // (1) unsynchronized read
> valBytes = ctx.marshal(val); // (2) unsynchronized write
> }
> {code}
> Two threads executing {{(1)}} concurrently both see {{null}}, both execute
> {{(2)}}. {{valBytes}} is not {{volatile}}, so publication is also unspecified
> — another thread reading the field after the race can observe {{null}} even
> after both writers completed on other cores.
> Same pattern in {{CacheObjectImpl.prepareMarshal}}.
> {{CacheObjectByteArrayImpl.prepareMarshal}} is a no-op and is unaffected.
> h2. Why it's a functional no-op today (but still a bug)
> * {{val}} is effectively immutable once the {{CacheObject}} is constructed.
> * {{ctx.marshal(val)}} is deterministic: equal inputs produce equal byte
> arrays.
> * Final read by the NIO writer goes through {{writeExternal}} / {{putValue}}
> / {{valueBytesLength}}, each of which calls {{valueBytes(ctx)}} that re-does
> the idempotency check and will populate {{valBytes}} on the calling thread if
> both racers somehow lost their writes.
> So the race does not corrupt on-wire output in practice. What it does cost:
> * *Wasted CPU*: duplicate {{Marshaller.marshal}} calls on contended keys (hot
> paths during tx prepare when the same key appears in multiple
> {{IgniteTxEntry}} instances).
> * *Wasted allocation*: each losing racer produces a {{byte[]}} that's
> immediately garbage.
> * *Debugging friction*: any future audit or assertion that presumes "marshal
> happens once per CO" cannot be added without first fixing this.
> * *Specification drift*: the contract is effectively "idempotent under
> concurrency by accident" — reliant on value immutability and marshaller
> determinism rather than stated.
> h2. Reproduction
> * Run {{IgniteCacheQueryMultiThreadedSelfTest}} with {{-ea}} after
> temporarily re-adding the assertion from the Phase 2 audit:
> {code:java}
> // in KeyCacheObjectImpl.prepareMarshal
> if (valBytes == null) {
> assert assertFirstPrepareMarshal();
> valBytes = ctx.marshal(val);
> }
> {code}
> Plus the transient flag + helper on {{CacheObjectAdapter}}.
> * The assertion fires from the {{IgniteTxEntry.marshal → key.prepareMarshal}}
> call site when a key is shared across concurrent prepare flows.
> h2. Proposed fix
> Synchronize the check-and-set. Two options, in order of preference:
> # *Double-checked locking on a per-instance monitor*, keeping the hot no-op
> branch lock-free:
> {code:java}
> @Override public void prepareMarshal(CacheObjectValueContext ctx) throws
> IgniteCheckedException {
> if (valBytes != null)
> return;
> synchronized (this) {
> if (valBytes == null)
> valBytes = ctx.marshal(val);
> }
> }
> {code}
> Requires making {{valBytes}} {{volatile}} for safe DCL. Uncontended cost: one
> volatile read. Contended: {{synchronized}} on the {{CacheObject}} itself,
> which is already the per-key identity.
> # *Atomic publish via {{VarHandle}} / {{Unsafe.compareAndSetObject}}* on
> {{valBytes}} — winner's bytes publish, loser's garbage. Avoids entering
> {{synchronized}}, but requires a VarHandle field and the losing thread still
> does the marshal work.
> Option 1 eliminates both races and duplicate work; option 2 eliminates the
> race but preserves duplicate work. Option 1 is recommended.
> Mirror the same treatment in {{CacheObjectImpl.prepareMarshal}} (which calls
> {{valueBytesFromValue(ctx)}} instead of {{ctx.marshal(val)}}).
> h2. Scope
> * {{modules/core}}:
> ** {{KeyCacheObjectImpl.prepareMarshal}}
> ** {{CacheObjectImpl.prepareMarshal}}
> ** {{CacheObjectAdapter.valBytes}} → {{volatile byte[]}}
> * No wire-protocol change (field layout unchanged).
> * No API change.
> h2. Acceptance criteria
> * Under {{-ea}}, the Phase 2 audit assertion re-added to
> {{KeyCacheObjectImpl.prepareMarshal}} / {{CacheObjectImpl.prepareMarshal}}
> does not fire in {{IgniteCacheQueryMultiThreadedSelfTest}} or other
> multi-threaded suites.
> * No measurable throughput regression on {{GridCachePutAllFailoverSelfTest}}
> and {{GridCacheRebalancingSyncSelfTest}} (these exercise the same CO marshal
> paths on the hot path).
> * Unit test: two threads race on a fresh
> {{KeyCacheObjectImpl.prepareMarshal}} — {{ctx.marshal(val)}} is invoked
> exactly once (verified via a counting {{CacheObjectValueContext}} mock).
--
This message was sent by Atlassian Jira
(v8.20.10#820010)