-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
Fix the potential race condition in the BlobStore readhandler #12123
Conversation
--- *Motivation* We found the BlobStoreBackedReadHandler enter an infinite loop when reading a offload ledger. We saw in the heap dump, there has two ledger 1 and 2, and we have a consumer is reading ledger 1, but the heap shows the buffer it reading is ledger 2. Then the read handler read a wrong entry id and the entry id is out of range between the firstEntryId and the lastEntryId, it will try to keep seeking to the right position, so that it enter an infinite loop. The buffer in the `BlobStoreBackedInputStreamImpl` is a wrong buffer, so the read handler can not read a right entry from it. After investigating, the buffer used in the `BlobStoreBackedInputStreamImpl` is allocated from `PulsarByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize)`, by default, we used a `PooledByteBufAllocator.DEFAULT` to allocate memory for that buffer, a weird thing we found is the closed buffer still can read data if there has new buffer allocated and write things. This is the test code: ``` ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); buf.writeByte(1); System.out.println(buf.readByte()); // print 1 buf.release(); //System.out.println(buf.readByte()); // will throw exception ByteBuf newBuf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); newBuf.writeByte(2); System.out.println(buf.readByte()); // print 2 newBuf.release(); //System.out.println(buf.readByte()); ``` So we suspect there has a race condition between the read and close operation. That cause there has a thread start reading after the read handler closed. *Modifications* - Add a state check before rading entries. - exit loop when the entryID is bigger than the lastEntryID
I will try to add test later |
@zymap Thanks for your contribution. For this PR, do we need to update docs? (The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks) |
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.
Great catch! It'll be good to add the test, as you mentioned @zymap.
State state = STATE_UPDATER.get(this); | ||
if (state == State.Closed) { | ||
log.warn("Reading a closed read handler. Ledger ID: {}, Read range: {}-{}", ledgerId, firstEntry, lastEntry); | ||
throw new BKException.BKUnexpectedConditionException(); |
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.
Should this complete the promise
exceptionally?
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 promise
is completed in the catch
section.
private AtomicReferenceFieldUpdater<BlobStoreBackedReadHandleImpl, State> STATE_UPDATER = AtomicReferenceFieldUpdater | ||
.newUpdater(BlobStoreBackedReadHandleImpl.class, State.class, "state"); | ||
private volatile State state = null; |
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'm not sure that we need these concurrency controls. The executor
in this class is a single thread. Therefore, a local state
variable should be sufficient as long as we only update the variable from within the executor
's thread.
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.
Yes. We did don't need the AtomicReferenceFieldUpdater
to control it. But we need to make it as volatile
, the read and close in the different thread, if the read thread into an infinite loop, then someone triggers the close, the read thread can not exit the infinite loop because the state is holding by the read thread. We need to add the volatile to make the state can be notified when it's changed.
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 read and close in the different thread, if the read thread into an infinite loop, then someone triggers the close, the read thread can not exit the infinite loop because the state is holding by the read thread.
The read and the write are not in separate threads. They are both scheduled to run as runnables on the executor
in this class. That executor
has a single thread. If the read is in an infinite loop, the close
runnable will never run.
07b0a4e
to
c01e7b9
Compare
c01e7b9
to
ac62b65
Compare
@zymap Please take a look of the failed tests
|
@michaeljmarshall @hangc0276 @codelipenghui All tests passed, PTAL, thanks |
} | ||
entriesToRead--; | ||
nextExpectedId++; | ||
} else if (entryId > nextExpectedId && entryId < lastEntry) { |
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 two if check are lead to
inputStream.seek(index.getIndexEntryForEntry(nextExpectedId).getDataOffset());
continue;
can we merge them into one if check?
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.
ok
@@ -94,56 +104,70 @@ public LedgerMetadata getLedgerMetadata() { | |||
log.debug("Ledger {}: reading {} - {}", getId(), firstEntry, lastEntry); | |||
CompletableFuture<LedgerEntries> promise = new CompletableFuture<>(); | |||
executor.submit(() -> { | |||
List<LedgerEntry> entries = new ArrayList<LedgerEntry>(); |
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.
We'd better put it behind
if (firstEntry > lastEntry
|| firstEntry < 0
|| lastEntry > getLastAddConfirmed()) {
promise.completeExceptionally(new BKException.BKIncorrectParameterException());
return;
}
we can save the list object apply when the (firstEntry, lastEntry) check failed.
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 array list will not allocate space until there is an element added to it. And it is used in the catch
, I think we can keep it.
@@ -115,7 +115,8 @@ public int read(byte[] b, int off, int len) throws IOException { | |||
} | |||
|
|||
@Override | |||
public void seek(long position) { | |||
public void seek(long position) throws IOException { | |||
refillBufferIfNeeded(); |
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.
Why add refillBufferIfNeed()
?
We'd better check buffer.readerIndex() == position
, if the check is true, just skip the following check and return.
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.
Because we add a new line in the BlobStoreBackedHandleImpl
:
inputStream.seek(index.getIndexEntryForEntry(firstEntry).getDataOffset());
Before this, the readAsync will use readInt() to trigger the stream to fill buffer, then to seek. Currently, we will execute seek before any read operation. We need to fill the buffer to let it fetch the data.
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 position
is the position of the whole object, and the buffer is part of the object. I think we don't need to check buffer.readerIndex() == position
, the seek operation is not a complicated work.
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.
It was added because my test case has some issues. Remove this and refactor the test case
nextExpectedId, entryId, lastEntry); | ||
throw new BKException.BKUnexpectedConditionException(); | ||
} else { | ||
long ignored = inputStream.skip(length); |
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 ignored
has no reference, we can use inputStream.skip(length);
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.
ok
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.
There has a spotbugs check to let us handle the returned value. So we can't remove it.
nextExpectedId, entryId, lastEntry); | ||
throw new BKException.BKUnexpectedConditionException(); | ||
} else { | ||
long ignored = inputStream.skip(length); |
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 ignored
has no reference, we can use inputStream.skip(length);
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.
ok
// will read the entry id from the stream and that is not the correct entry id, so it will seek to the | ||
// correct position then read the stream as normal. But the entry id may exceed the last entry id, that | ||
// will cause we are hardly to know the edge of the request range. | ||
inputStream.seek(index.getIndexEntryForEntry(firstEntry).getDataOffset()); |
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.
Because we have seek to the firstEntry before reading, we can simplify the following check logic. If the entryId read from dataStream not equal to nextExpectedId
, we can throw exception.
@codelipenghui Please help check this logic.
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.
This seek operation is being used to seeking to the first entry id position, and then do the original logic. This operation is only to make sure the read position is as we wanted.
Before this, it reads the first data to get to know if the read position is the right position. If not, it will seek to the right position.
The following check logic is still needed to help us to seek to the right position if we met some unexpected data. Maybe we can set a limitation to avoid it into an infinite loop.
@zymap - If I understand this class correctly, the new |
@michaeljmarshall Thanks, that makes sense to me. I will fix that |
@michaeljmarshall I removed the |
ping @hangc0276 @michaeljmarshall Please take another look when you have time. Thanks |
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.
Great Catch!
@michaeljmarshall Please help review the PR again |
@codelipenghui and @zymap - sorry for my delayed review. LGTM |
* Fix the potential race condition in the BlobStore readhandler --- *Motivation* We found the BlobStoreBackedReadHandler enter an infinite loop when reading a offload ledger. We saw in the heap dump, there has two ledger 1 and 2, and we have a consumer is reading ledger 1, but the heap shows the buffer it reading is ledger 2. Then the read handler read a wrong entry id and the entry id is out of range between the firstEntryId and the lastEntryId, it will try to keep seeking to the right position, so that it enter an infinite loop. The buffer in the `BlobStoreBackedInputStreamImpl` is a wrong buffer, so the read handler can not read a right entry from it. After investigating, the buffer used in the `BlobStoreBackedInputStreamImpl` is allocated from `PulsarByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize)`, by default, we used a `PooledByteBufAllocator.DEFAULT` to allocate memory for that buffer, a weird thing we found is the closed buffer still can read data if there has new buffer allocated and write things. This is the test code: ``` ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); buf.writeByte(1); System.out.println(buf.readByte()); // print 1 buf.release(); //System.out.println(buf.readByte()); // will throw exception ByteBuf newBuf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); newBuf.writeByte(2); System.out.println(buf.readByte()); // print 2 newBuf.release(); //System.out.println(buf.readByte()); ``` So we suspect there has a race condition between the read and close operation. That cause there has a thread start reading after the read handler closed. *Modifications* - Add a state check before reading entries. - exit loop when the entryID is bigger than the lastEntryID (cherry picked from commit 3d7c3f8)
--- *Motivation* In the apache#12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream.
--- *Motivation* In the apache#12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream.
--- *Motivation* In the #12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream.
--- *Motivation* In the #12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream. (cherry picked from commit b4d05ac)
--- *Motivation* In the apache#12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream.
* Fix the potential race condition in the BlobStore readhandler --- *Motivation* We found the BlobStoreBackedReadHandler enter an infinite loop when reading a offload ledger. We saw in the heap dump, there has two ledger 1 and 2, and we have a consumer is reading ledger 1, but the heap shows the buffer it reading is ledger 2. Then the read handler read a wrong entry id and the entry id is out of range between the firstEntryId and the lastEntryId, it will try to keep seeking to the right position, so that it enter an infinite loop. The buffer in the `BlobStoreBackedInputStreamImpl` is a wrong buffer, so the read handler can not read a right entry from it. After investigating, the buffer used in the `BlobStoreBackedInputStreamImpl` is allocated from `PulsarByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize)`, by default, we used a `PooledByteBufAllocator.DEFAULT` to allocate memory for that buffer, a weird thing we found is the closed buffer still can read data if there has new buffer allocated and write things. This is the test code: ``` ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); buf.writeByte(1); System.out.println(buf.readByte()); // print 1 buf.release(); //System.out.println(buf.readByte()); // will throw exception ByteBuf newBuf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); newBuf.writeByte(2); System.out.println(buf.readByte()); // print 2 newBuf.release(); //System.out.println(buf.readByte()); ``` So we suspect there has a race condition between the read and close operation. That cause there has a thread start reading after the read handler closed. *Modifications* - Add a state check before reading entries. - exit loop when the entryID is bigger than the lastEntryID (cherry picked from commit 3d7c3f8)
--- *Motivation* In the #12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream. (cherry picked from commit b4d05ac)
…#12123) * Fix the potential race condition in the BlobStore readhandler --- *Motivation* We found the BlobStoreBackedReadHandler enter an infinite loop when reading a offload ledger. We saw in the heap dump, there has two ledger 1 and 2, and we have a consumer is reading ledger 1, but the heap shows the buffer it reading is ledger 2. Then the read handler read a wrong entry id and the entry id is out of range between the firstEntryId and the lastEntryId, it will try to keep seeking to the right position, so that it enter an infinite loop. The buffer in the `BlobStoreBackedInputStreamImpl` is a wrong buffer, so the read handler can not read a right entry from it. After investigating, the buffer used in the `BlobStoreBackedInputStreamImpl` is allocated from `PulsarByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize)`, by default, we used a `PooledByteBufAllocator.DEFAULT` to allocate memory for that buffer, a weird thing we found is the closed buffer still can read data if there has new buffer allocated and write things. This is the test code: ``` ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); buf.writeByte(1); System.out.println(buf.readByte()); // print 1 buf.release(); //System.out.println(buf.readByte()); // will throw exception ByteBuf newBuf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); newBuf.writeByte(2); System.out.println(buf.readByte()); // print 2 newBuf.release(); //System.out.println(buf.readByte()); ``` So we suspect there has a race condition between the read and close operation. That cause there has a thread start reading after the read handler closed. *Modifications* - Add a state check before reading entries. - exit loop when the entryID is bigger than the lastEntryID
…#12123) * Fix the potential race condition in the BlobStore readhandler --- *Motivation* We found the BlobStoreBackedReadHandler enter an infinite loop when reading a offload ledger. We saw in the heap dump, there has two ledger 1 and 2, and we have a consumer is reading ledger 1, but the heap shows the buffer it reading is ledger 2. Then the read handler read a wrong entry id and the entry id is out of range between the firstEntryId and the lastEntryId, it will try to keep seeking to the right position, so that it enter an infinite loop. The buffer in the `BlobStoreBackedInputStreamImpl` is a wrong buffer, so the read handler can not read a right entry from it. After investigating, the buffer used in the `BlobStoreBackedInputStreamImpl` is allocated from `PulsarByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize)`, by default, we used a `PooledByteBufAllocator.DEFAULT` to allocate memory for that buffer, a weird thing we found is the closed buffer still can read data if there has new buffer allocated and write things. This is the test code: ``` ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); buf.writeByte(1); System.out.println(buf.readByte()); // print 1 buf.release(); //System.out.println(buf.readByte()); // will throw exception ByteBuf newBuf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); newBuf.writeByte(2); System.out.println(buf.readByte()); // print 2 newBuf.release(); //System.out.println(buf.readByte()); ``` So we suspect there has a race condition between the read and close operation. That cause there has a thread start reading after the read handler closed. *Modifications* - Add a state check before reading entries. - exit loop when the entryID is bigger than the lastEntryID (cherry picked from commit 3d7c3f8)
--- *Motivation* In the apache#12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream. (cherry picked from commit b4d05ac)
…#12123) * Fix the potential race condition in the BlobStore readhandler --- *Motivation* We found the BlobStoreBackedReadHandler enter an infinite loop when reading a offload ledger. We saw in the heap dump, there has two ledger 1 and 2, and we have a consumer is reading ledger 1, but the heap shows the buffer it reading is ledger 2. Then the read handler read a wrong entry id and the entry id is out of range between the firstEntryId and the lastEntryId, it will try to keep seeking to the right position, so that it enter an infinite loop. The buffer in the `BlobStoreBackedInputStreamImpl` is a wrong buffer, so the read handler can not read a right entry from it. After investigating, the buffer used in the `BlobStoreBackedInputStreamImpl` is allocated from `PulsarByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize)`, by default, we used a `PooledByteBufAllocator.DEFAULT` to allocate memory for that buffer, a weird thing we found is the closed buffer still can read data if there has new buffer allocated and write things. This is the test code: ``` ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); buf.writeByte(1); System.out.println(buf.readByte()); // print 1 buf.release(); //System.out.println(buf.readByte()); // will throw exception ByteBuf newBuf = PulsarByteBufAllocator.DEFAULT.buffer(1024, 1024); newBuf.writeByte(2); System.out.println(buf.readByte()); // print 2 newBuf.release(); //System.out.println(buf.readByte()); ``` So we suspect there has a race condition between the read and close operation. That cause there has a thread start reading after the read handler closed. *Modifications* - Add a state check before reading entries. - exit loop when the entryID is bigger than the lastEntryID (cherry picked from commit 3d7c3f8)
--- *Motivation* In the apache#12123, I add the seek operation at the readAsync method. It makes sure the data stream always seek to the first entry position to read and will not introduce EOF exception. But in the offload index entry, it groups a set of entries into a range, the seek operation will seek the posistion to the first entry in the range. That will introduce a performance issue because every read opeartion will read from the first entry in the range until it find the actual first read entry. But if we remove the seek operation, that will cause a EOF exception from the readAsync method. This PR adds a limitation of the seek opeartion. *Modifications* Add available method in the backedInputStream to get know how many bytes we can read from the stream. (cherry picked from commit b4d05ac)
Motivation
We found the BlobStoreBackedReadHandler enter an infinite loop when reading a offload ledger.
We saw in the heap dump, there has two ledger 1 and 2, and we have
a consumer is reading ledger 1, but the heap shows the buffer it
reading is ledger 2. Then the read handler read a wrong entry id
and the entry id is out of range between the firstEntryId and the
lastEntryId, it will try to keep seeking to the right position, so
that it enters an infinite loop.
The buffer in the
BlobStoreBackedInputStreamImpl
is a wrong buffer,so the read handler can not read a correct entry from it. After investigating,
the buffer used in the
BlobStoreBackedInputStreamImpl
is allocated fromPulsarByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize)
, by default,we used a
PooledByteBufAllocator.DEFAULT
to allocate memory for that buffer,a weird thing we found is the closed buffer still can read data if there has
new buffer allocated and write things.
This is the test code:
So we suspect there has a race condition between the read and close
operation. That cause there has a thread start reading after
the read handler closed.
Modifications