-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[fix][broker] Fixed error when delayed messages trackers state grows to >1.5GB #16490
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
Merged
Merged
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
e4a69d7
Fixed error when delayed messages trackers state grows to >1.5GB
merlimat 5065e9a
Fixed spotbugs issues
merlimat 8b0dcca
Fixed javadocs
merlimat e82597c
In the constructor, ensure all segments after the first one are of ma…
merlimat 111d2c4
Merge remote-tracking branch 'apache/master' into fix-delay-tracker
merlimat 9b5c62b
Use poll to figure out where the test is stuck
merlimat 42568f2
Added SegmentedLongArray specific unit test
merlimat 18a6cac
Merge remote-tracking branch 'apache/master' into fix-delay-tracker
merlimat 01ca472
Removed unused imports
merlimat File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
128 changes: 128 additions & 0 deletions
128
...ar-common/src/main/java/org/apache/pulsar/common/util/collections/SegmentedLongArray.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,128 @@ | ||
| /** | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.pulsar.common.util.collections; | ||
|
|
||
| import io.netty.buffer.ByteBuf; | ||
| import io.netty.buffer.PooledByteBufAllocator; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import javax.annotation.concurrent.NotThreadSafe; | ||
| import lombok.Getter; | ||
|
|
||
| @NotThreadSafe | ||
| public class SegmentedLongArray implements AutoCloseable { | ||
|
|
||
| private static final int SIZE_OF_LONG = 8; | ||
|
|
||
| private static final int MAX_SEGMENT_SIZE = 2 * 1024 * 1024; // 2M longs -> 16 MB | ||
| private final List<ByteBuf> buffers = new ArrayList<>(); | ||
|
|
||
| @Getter | ||
| private final long initialCapacity; | ||
|
|
||
| @Getter | ||
| private long capacity; | ||
|
|
||
| public SegmentedLongArray(long initialCapacity) { | ||
| long remainingToAdd = initialCapacity; | ||
|
|
||
| // Add first segment | ||
| int sizeToAdd = (int) Math.min(remainingToAdd, MAX_SEGMENT_SIZE); | ||
| ByteBuf buffer = PooledByteBufAllocator.DEFAULT.directBuffer(sizeToAdd * SIZE_OF_LONG); | ||
| buffer.writerIndex(sizeToAdd * SIZE_OF_LONG); | ||
| buffers.add(buffer); | ||
| remainingToAdd -= sizeToAdd; | ||
|
|
||
| // Add the remaining segments, all at full segment size, if necessary | ||
| while (remainingToAdd > 0) { | ||
| buffer = PooledByteBufAllocator.DEFAULT.directBuffer(MAX_SEGMENT_SIZE * SIZE_OF_LONG); | ||
| buffer.writerIndex(MAX_SEGMENT_SIZE * SIZE_OF_LONG); | ||
| buffers.add(buffer); | ||
| remainingToAdd -= MAX_SEGMENT_SIZE; | ||
| } | ||
|
|
||
| this.initialCapacity = initialCapacity; | ||
| this.capacity = this.initialCapacity; | ||
| } | ||
|
|
||
| public void writeLong(long offset, long value) { | ||
| int bufferIdx = (int) (offset / MAX_SEGMENT_SIZE); | ||
| int internalIdx = (int) (offset % MAX_SEGMENT_SIZE); | ||
| buffers.get(bufferIdx).setLong(internalIdx * SIZE_OF_LONG, value); | ||
| } | ||
|
|
||
| public long readLong(long offset) { | ||
| int bufferIdx = (int) (offset / MAX_SEGMENT_SIZE); | ||
| int internalIdx = (int) (offset % MAX_SEGMENT_SIZE); | ||
| return buffers.get(bufferIdx).getLong(internalIdx * SIZE_OF_LONG); | ||
| } | ||
|
|
||
| public void increaseCapacity() { | ||
| if (capacity < MAX_SEGMENT_SIZE) { | ||
| // Resize the current buffer to bigger capacity | ||
| capacity += (capacity <= 256 ? capacity : capacity / 2); | ||
| capacity = Math.min(capacity, MAX_SEGMENT_SIZE); | ||
| buffers.get(0).capacity((int) this.capacity * SIZE_OF_LONG); | ||
| buffers.get(0).writerIndex((int) this.capacity * SIZE_OF_LONG); | ||
| } else { | ||
| // Let's add 1 mode buffer to the list | ||
| int bufferSize = MAX_SEGMENT_SIZE * SIZE_OF_LONG; | ||
| ByteBuf buffer = PooledByteBufAllocator.DEFAULT.directBuffer(bufferSize, bufferSize); | ||
| buffer.writerIndex(bufferSize); | ||
| buffers.add(buffer); | ||
| capacity += MAX_SEGMENT_SIZE; | ||
| } | ||
| } | ||
|
|
||
| public void shrink(long newCapacity) { | ||
| if (newCapacity >= capacity || newCapacity < initialCapacity) { | ||
| return; | ||
| } | ||
|
|
||
| long sizeToReduce = capacity - newCapacity; | ||
| while (sizeToReduce >= MAX_SEGMENT_SIZE && buffers.size() > 1) { | ||
| ByteBuf b = buffers.remove(buffers.size() - 1); | ||
| b.release(); | ||
| capacity -= MAX_SEGMENT_SIZE; | ||
| sizeToReduce -= MAX_SEGMENT_SIZE; | ||
| } | ||
|
|
||
| if (buffers.size() == 1 && sizeToReduce > 0) { | ||
| // We should also reduce the capacity of the first buffer | ||
| capacity -= sizeToReduce; | ||
| ByteBuf oldBuffer = buffers.get(0); | ||
| ByteBuf newBuffer = PooledByteBufAllocator.DEFAULT.directBuffer((int) capacity * SIZE_OF_LONG); | ||
| oldBuffer.getBytes(0, newBuffer, (int) capacity * SIZE_OF_LONG); | ||
| oldBuffer.release(); | ||
| buffers.set(0, newBuffer); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| buffers.forEach(ByteBuf::release); | ||
| } | ||
|
|
||
| /** | ||
| * The amount of memory used to back the array of longs. | ||
| */ | ||
| public long bytesCapacity() { | ||
| return capacity * SIZE_OF_LONG; | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 size of the last segment might also < MAX_SEGMENT_SIZE, we need to consider to update the capacity of the buffer?
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.
And the last segment size < MAX_SEGMENT_SIZE will also affect the
writeLongandreadLongmethod because we always uses theMAX_SEGMENT_SIZEto calculate the index.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.
All the segments, after the first one, will be created directly at
MAX_SEGMENT_SIZEand will not get expanded/shrinked, only added or removed.Uh oh!
There was an error while loading. Please reload this page.
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.
I think here is possible to add a buffer which is not with
MAX_SEGMENT_SIZE? https://github.com/apache/pulsar/pull/16490/files#diff-ee5391a05253205e8b9bb0f52faa7397326ec0687513204b3bf1b5244d2b8504R45-R50There 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, that's true! I need to fix the constructor
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.
Fixed