[
https://issues.apache.org/jira/browse/HDDS-15341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Siyao Meng updated HDDS-15341:
------------------------------
Description:
In Ozone, this can be hit when multiple EC key output streams in the same
client JVM use the Java raw EC encoder concurrently with different encode/reset
lengths. Each ECKeyOutputStream has its own encoder, but all Java encoders
share the static CoderUtil.emptyChunk cache. *If native ISA-L is unavailable or
not selected*, the Java RSRawEncoder clears parity output buffers through
CoderUtil.resetOutputBuffers(). Under concurrent close/flush paths, especially
with partial final stripes of different sizes, one stream can grow the shared
zero buffer for a larger encode while another smaller encode races and shrinks
it, causing the larger encode’s later System.arraycopy() to throw
ArrayIndexOutOfBoundsException.
Problem:
CoderUtil.resetBuffer(byte[] buffer, int offset, int len) gets a shared
zero-filled buffer from getEmptyChunk(len) and then calls:
{code}
System.arraycopy(empty, 0, buffer, offset, len);
{code}
The old getEmptyChunk() implementation checked emptyChunk.length before
entering the synchronized block, unconditionally replaced the shared static
buffer inside the lock, and returned the shared static field after leaving the
lock. This allowed a smaller concurrent caller to shrink the shared cached
buffer after a larger caller had grown it.
{code:title=Stacktrace}
ArrayIndexOutOfBoundsException: java.lang.ArrayIndexOutOfBoundsException
at java.lang.System.arraycopy(Native Method)
at
org.apache.ozone.erasurecode.rawcoder.CoderUtil.resetBuffer(CoderUtil.java:76)
at
org.apache.ozone.erasurecode.rawcoder.CoderUtil.resetOutputBuffers(CoderUtil.java:96)
at
org.apache.ozone.erasurecode.rawcoder.RSRawEncoder.doEncode(RSRawEncoder.java:69)
at
org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder.encode(RawErasureEncoder.java:88)
at
org.apache.hadoop.ozone.client.io.ECKeyOutputStream.generateParityCells(ECKeyOutputStream.java:305)
at
org.apache.hadoop.ozone.client.io.ECKeyOutputStream.close(ECKeyOutputStream.java:475)
at
org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:105)
at
org.apache.hadoop.fs.ozone.OzoneFSOutputStream.close(OzoneFSOutputStream.java:70)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:77)
at
org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
{code}
{code:title=Current logic w/o the fix}
static byte[] getEmptyChunk(int leastLength) {
if (emptyChunk.length >= leastLength) {
return emptyChunk; // In most time
}
synchronized (CoderUtil.class) {
emptyChunk = new byte[leastLength];
}
{code}
Repro:
1. emptyChunk starts as byte[4096].
2. Thread A calls getEmptyChunk(4097) and blocks before entering the
synchronized block.
3. Thread B calls getEmptyChunk(8194), enters the synchronized block, and sets
emptyChunk = byte[8194].
4. Thread A resumes and unconditionally sets emptyChunk = byte[4097].
5. Thread B returns the shared static emptyChunk, now byte[4097].
6. System.arraycopy(..., len=8194) throws ArrayIndexOutOfBoundsException.
This is a TOCTOU-style race on the shared emptyChunk cache.
was:
In Ozone, this can be hit when multiple EC key output streams in the same
client JVM use the Java raw EC encoder concurrently with different encode/reset
lengths. Each ECKeyOutputStream has its own encoder, but all Java encoders
share the static CoderUtil.emptyChunk cache. If native ISA-L is unavailable or
not selected, the Java RSRawEncoder clears parity output buffers through
CoderUtil.resetOutputBuffers(). Under concurrent close/flush paths, especially
with partial final stripes of different sizes, one stream can grow the shared
zero buffer for a larger encode while another smaller encode races and shrinks
it, causing the larger encode’s later System.arraycopy() to throw
ArrayIndexOutOfBoundsException.
Problem:
CoderUtil.resetBuffer(byte[] buffer, int offset, int len) gets a shared
zero-filled buffer from getEmptyChunk(len) and then calls:
{code}
System.arraycopy(empty, 0, buffer, offset, len);
{code}
The old getEmptyChunk() implementation checked emptyChunk.length before
entering the synchronized block, unconditionally replaced the shared static
buffer inside the lock, and returned the shared static field after leaving the
lock. This allowed a smaller concurrent caller to shrink the shared cached
buffer after a larger caller had grown it.
{code:title=Stacktrace}
ArrayIndexOutOfBoundsException: java.lang.ArrayIndexOutOfBoundsException
at java.lang.System.arraycopy(Native Method)
at
org.apache.ozone.erasurecode.rawcoder.CoderUtil.resetBuffer(CoderUtil.java:76)
at
org.apache.ozone.erasurecode.rawcoder.CoderUtil.resetOutputBuffers(CoderUtil.java:96)
at
org.apache.ozone.erasurecode.rawcoder.RSRawEncoder.doEncode(RSRawEncoder.java:69)
at
org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder.encode(RawErasureEncoder.java:88)
at
org.apache.hadoop.ozone.client.io.ECKeyOutputStream.generateParityCells(ECKeyOutputStream.java:305)
at
org.apache.hadoop.ozone.client.io.ECKeyOutputStream.close(ECKeyOutputStream.java:475)
at
org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:105)
at
org.apache.hadoop.fs.ozone.OzoneFSOutputStream.close(OzoneFSOutputStream.java:70)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:77)
at
org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
{code}
{code:title=Current logic w/o the fix}
static byte[] getEmptyChunk(int leastLength) {
if (emptyChunk.length >= leastLength) {
return emptyChunk; // In most time
}
synchronized (CoderUtil.class) {
emptyChunk = new byte[leastLength];
}
{code}
Repro:
1. emptyChunk starts as byte[4096].
2. Thread A calls getEmptyChunk(4097) and blocks before entering the
synchronized block.
3. Thread B calls getEmptyChunk(8194), enters the synchronized block, and sets
emptyChunk = byte[8194].
4. Thread A resumes and unconditionally sets emptyChunk = byte[4097].
5. Thread B returns the shared static emptyChunk, now byte[4097].
6. System.arraycopy(..., len=8194) throws ArrayIndexOutOfBoundsException.
This is a TOCTOU-style race on the shared emptyChunk cache.
> EC client write can fail with ArrayIndexOutOfBoundsException due to CoderUtil
> emptyChunk resize race
> ----------------------------------------------------------------------------------------------------
>
> Key: HDDS-15341
> URL: https://issues.apache.org/jira/browse/HDDS-15341
> Project: Apache Ozone
> Issue Type: Bug
> Components: EC Client
> Reporter: Siyao Meng
> Assignee: Siyao Meng
> Priority: Major
> Labels: pull-request-available
>
> In Ozone, this can be hit when multiple EC key output streams in the same
> client JVM use the Java raw EC encoder concurrently with different
> encode/reset lengths. Each ECKeyOutputStream has its own encoder, but all
> Java encoders share the static CoderUtil.emptyChunk cache. *If native ISA-L
> is unavailable or not selected*, the Java RSRawEncoder clears parity output
> buffers through CoderUtil.resetOutputBuffers(). Under concurrent close/flush
> paths, especially with partial final stripes of different sizes, one stream
> can grow the shared zero buffer for a larger encode while another smaller
> encode races and shrinks it, causing the larger encode’s later
> System.arraycopy() to throw ArrayIndexOutOfBoundsException.
> Problem:
> CoderUtil.resetBuffer(byte[] buffer, int offset, int len) gets a shared
> zero-filled buffer from getEmptyChunk(len) and then calls:
> {code}
> System.arraycopy(empty, 0, buffer, offset, len);
> {code}
> The old getEmptyChunk() implementation checked emptyChunk.length before
> entering the synchronized block, unconditionally replaced the shared static
> buffer inside the lock, and returned the shared static field after leaving
> the lock. This allowed a smaller concurrent caller to shrink the shared
> cached buffer after a larger caller had grown it.
> {code:title=Stacktrace}
> ArrayIndexOutOfBoundsException: java.lang.ArrayIndexOutOfBoundsException
> at java.lang.System.arraycopy(Native Method)
> at
> org.apache.ozone.erasurecode.rawcoder.CoderUtil.resetBuffer(CoderUtil.java:76)
> at
> org.apache.ozone.erasurecode.rawcoder.CoderUtil.resetOutputBuffers(CoderUtil.java:96)
> at
> org.apache.ozone.erasurecode.rawcoder.RSRawEncoder.doEncode(RSRawEncoder.java:69)
> at
> org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder.encode(RawErasureEncoder.java:88)
> at
> org.apache.hadoop.ozone.client.io.ECKeyOutputStream.generateParityCells(ECKeyOutputStream.java:305)
> at
> org.apache.hadoop.ozone.client.io.ECKeyOutputStream.close(ECKeyOutputStream.java:475)
> at
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:105)
> at
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.close(OzoneFSOutputStream.java:70)
> at
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:77)
> at
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
> {code}
> {code:title=Current logic w/o the fix}
> static byte[] getEmptyChunk(int leastLength) {
> if (emptyChunk.length >= leastLength) {
> return emptyChunk; // In most time
> }
> synchronized (CoderUtil.class) {
> emptyChunk = new byte[leastLength];
> }
> {code}
> Repro:
> 1. emptyChunk starts as byte[4096].
> 2. Thread A calls getEmptyChunk(4097) and blocks before entering the
> synchronized block.
> 3. Thread B calls getEmptyChunk(8194), enters the synchronized block, and
> sets emptyChunk = byte[8194].
> 4. Thread A resumes and unconditionally sets emptyChunk = byte[4097].
> 5. Thread B returns the shared static emptyChunk, now byte[4097].
> 6. System.arraycopy(..., len=8194) throws ArrayIndexOutOfBoundsException.
> This is a TOCTOU-style race on the shared emptyChunk cache.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]