Skip to content

Comments

Fix buffer capacity race condition in spatial#16931

Merged
clintropolis merged 4 commits intoapache:masterfrom
pranavbhole:spatialRace
Aug 27, 2024
Merged

Fix buffer capacity race condition in spatial#16931
clintropolis merged 4 commits intoapache:masterfrom
pranavbhole:spatialRace

Conversation

@pranavbhole
Copy link
Contributor

@pranavbhole pranavbhole commented Aug 21, 2024

Description

This PR has potential fix for the following exception, found that we are creating immutable node and passing the mutable data buffer to immutable node.

java.lang.IllegalArgumentException: newLimit > capacity: (20 > 18)
	at java.base/java.nio.Buffer.createLimitException(Buffer.java:372) ~[?:?]
	at java.base/java.nio.Buffer.limit(Buffer.java:346) ~[?:?]
	at java.base/java.nio.ByteBuffer.limit(ByteBuffer.java:1107) ~[?:?]
	at java.base/java.nio.MappedByteBuffer.limit(MappedByteBuffer.java:235) ~[?:?]
	at java.base/java.nio.MappedByteBuffer.limit(MappedByteBuffer.java:67) ~[?:?]
	at org.roaringbitmap.buffer.ImmutableRoaringArray.<init>(ImmutableRoaringArray.java:52) ~[RoaringBitmap-0.9.49.jar:?]
	at org.roaringbitmap.buffer.ImmutableRoaringBitmap.<init>(ImmutableRoaringBitmap.java:1057) ~[RoaringBitmap-0.9.49.jar:?]
	at org.apache.druid.collections.bitmap.WrappedImmutableRoaringBitmap.<init>(WrappedImmutableRoaringBitmap.java:38) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.apache.druid.collections.bitmap.RoaringBitmapFactory.mapImmutableBitmap(RoaringBitmapFactory.java:126) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.apache.druid.collections.spatial.ImmutableFloatNode.getImmutableBitmap(ImmutableFloatNode.java:155) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.apache.druid.collections.spatial.search.GutmanSearchStrategy$2.apply(GutmanSearchStrategy.java:57) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.apache.druid.collections.spatial.search.GutmanSearchStrategy$2.apply(GutmanSearchStrategy.java:53) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at com.google.common.collect.Iterators$6.transform(Iterators.java:828) ~[guava-32.0.1-jre.jar:?]
	at com.google.common.collect.TransformedIterator.next(TransformedIterator.java:52) ~[guava-32.0.1-jre.jar:?]
	at org.apache.druid.collections.bitmap.RoaringBitmapFactory$1$1.next(RoaringBitmapFactory.java:84) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.apache.druid.collections.bitmap.RoaringBitmapFactory$1$1.next(RoaringBitmapFactory.java:68) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.roaringbitmap.buffer.BufferFastAggregation.naive_or(BufferFastAggregation.java:782) ~[RoaringBitmap-0.9.49.jar:?]
	at org.roaringbitmap.buffer.BufferFastAggregation.or(BufferFastAggregation.java:875) ~[RoaringBitmap-0.9.49.jar:?]
	at org.roaringbitmap.buffer.ImmutableRoaringBitmap.or(ImmutableRoaringBitmap.java:875) ~[RoaringBitmap-0.9.49.jar:?]
	at org.apache.druid.collections.bitmap.RoaringBitmapFactory.union(RoaringBitmapFactory.java:142) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.apache.druid.query.DefaultBitmapResultFactory.unionDimensionValueBitmaps(DefaultBitmapResultFactory.java:73) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]
	at org.apache.druid.query.DefaultBitmapResultFactory.unionDimensionValueBitmaps(DefaultBitmapResultFactory.java:25) ~[druid-processing-2024.07.0-iap.jar:2024.07.0-iap]


This PR has:

  • been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • a release note entry in the PR description.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

initialOffset,
data.getInt(childrenOffset + (count++) * Integer.BYTES),
data,
data.duplicate(),
Copy link
Member

@clintropolis clintropolis Aug 21, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it looks like most of the usages of ByteBuffer by ImmutableFloatNode use calls that specify the offset in the buffer except for getImmutableBitmap(), which sets the position and slices it (for some reason, it seems like this method in general could be cleaned up). I wonder if we just change that instead of calling duplicate inside the iterator, so that way it is a bit lazier?

  @Override
  public ImmutableBitmap getImmutableBitmap()
  {
    final int sizePosition = initialOffset + offsetFromInitial + HEADER_NUM_BYTES + 2 * numDims * Float.BYTES;
    int numBytes = data.getInt(sizePosition);
    final ByteBuffer readOnlyBuffer = data.asReadOnlyBuffer();
    readOnlyBuffer.position(sizePosition + Integer.BYTES);
    readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
    return bitmapFactory.mapImmutableBitmap(readOnlyBuffer);
  }

This is consistent with how getCoords works, which also does something similar of making a readonly copy.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Though the current way, also avoids any potential misuse of the buffer in this class.

Copy link
Member

@clintropolis clintropolis Aug 21, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yea, but it would call duplicate for every entry instead of just the ones that satisfy the bounds check that we decided to use the bitmap

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clintropolis Thank you Clint, I liked the suggestion as well as it can avoid creating duplicate references on each tree node and sometimes tree can be really big.

@pranavbhole
Copy link
Contributor Author

@clintropolis @abhishekagarwal87 Can you please review the additional instrumentations?
Thank you

Comment on lines 160 to 166
log.error(e, "Failed to read bitmap from buffer '%s',"
+ "current pos: %d, set pos: %d, set limit: %d",
Base64.getEncoder().encodeToString(readOnlyBuffer.array()),
readOnlyBuffer.position(),
newPosition,
newLimit
);
Copy link
Member

@clintropolis clintropolis Aug 23, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, if buffer is mmap file then i think array() will not work, afaik it only works for buffers that wrap a byte[]. Also is this log message displaying the buffer contents that useful? Why does this need to be in a try/catch at all?

@pranavbhole
Copy link
Contributor Author

pranavbhole commented Aug 23, 2024

As per discussion with Clint, instrumentation is not really helpful here. We can not print buffer as it can be really big (roughly speaking sometimes 100MB as well).
cc @clintropolis @abhishekagarwal87

Can you please +1 ?

@abhishekagarwal87
Copy link
Contributor

I really wasn’t suggesting to print the whole buffer. We know that the code think it needs to read 20 bytes while only 18 were left. So we can print those 18 bytes and can inspect that.

One other thing we can also do is copy the offending sequence to a temporary location so it can be downloaded for offline inspection.

@clintropolis clintropolis merged commit 0caf383 into apache:master Aug 27, 2024
hevansDev pushed a commit to hevansDev/druid that referenced this pull request Aug 29, 2024
edgar2020 pushed a commit to edgar2020/druid that referenced this pull request Sep 5, 2024
@kfaraz kfaraz added this to the 31.0.0 milestone Oct 4, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants