-
Notifications
You must be signed in to change notification settings - Fork 892
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
Issue 550: add readLastAddConfirmedAndEntry in ReadHandle for long poll read #729
Changes from 3 commits
5eacce3
aa56e73
60eb76b
2cbe1ac
f67b40f
4eee6b1
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,6 +21,7 @@ | |
package org.apache.bookkeeper.client.api; | ||
|
||
import java.util.concurrent.CompletableFuture; | ||
import lombok.Data; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. you don't this import There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. right. will remove it. |
||
|
||
/** | ||
* Provide read access to a ledger. | ||
|
@@ -111,4 +112,32 @@ public interface ReadHandle extends Handle { | |
*/ | ||
long getLength(); | ||
|
||
/** | ||
* The type contains LAC and a LedgerEntry want to read. | ||
* It is used for readLastAddConfirmedAndEntry. | ||
*/ | ||
@Data | ||
class LastConfirmedAndEntry { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it is better that this class is not an inner class but a top level class There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. move it out of ReadHandle.java, make it an interface
create a class LastConfirmedAndEntryImpl to implement the interface. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks @eolivelli @sijie , will change it. |
||
private final Long lac; | ||
private final LedgerEntry entry; | ||
} | ||
|
||
/** | ||
* Asynchronous read specific entry and the latest last add confirmed. | ||
* If the next entryId is less than known last add confirmed, the call will read next entry directly. | ||
* If the next entryId is ahead of known last add confirmed, the call will issue a long poll read | ||
* to wait for the next entry <i>entryId</i>. | ||
* | ||
* @param entryId | ||
* next entry id to read | ||
* @param timeOutInMillis | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. in case of THIS timeout the CompletableFuture will return null ? or is will be completed exceptionally ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it will return There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. In LedgerHandle, most of the similar situation will return null. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we add a line which explains? That was my real request, sorry There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we add a line which explains? That was my real request, sorry There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks. sure will add it. |
||
* timeout period to wait for the entry id to be available (for long poll only) | ||
* @param parallel | ||
* whether to issue the long poll reads in parallel | ||
* @return an handle to the result of the operation | ||
*/ | ||
CompletableFuture<LastConfirmedAndEntry> readLastAddConfirmedAndEntry(long entryId, | ||
long timeOutInMillis, | ||
boolean parallel); | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -203,6 +203,12 @@ public void testOpenLedgerRead() throws Exception { | |
assertEquals(2, result(reader.tryReadLastAddConfirmed()).intValue()); | ||
checkEntries(result(reader.read(0, reader.getLastAddConfirmed())), data); | ||
checkEntries(result(reader.readUnconfirmed(0, reader.getLastAddConfirmed())), data); | ||
|
||
// test readLastAddConfirmedAndEntry | ||
ReadHandle.LastConfirmedAndEntry lastConfirmedAndEntry = | ||
result(reader.readLastAddConfirmedAndEntry(0, 999, false)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We have a 999 timeout less then 1 second. This test could be flaky. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @eolivelli Thanks. |
||
assertEquals(2, lastConfirmedAndEntry.getLac().intValue()); | ||
assertArrayEquals(data, lastConfirmedAndEntry.getEntry().getEntry()); | ||
} | ||
} | ||
|
||
|
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 is not needed.
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.
Thanks. It is originally there, this change adjust the order of importing, will handle it.