-
Notifications
You must be signed in to change notification settings - Fork 214
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
Add ProgressCheck callbacks to end-to-end acknowledgements #3565
Merged
Merged
Changes from 5 commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
21f3526
Add ProgressCheck callbacks to end-to-end acknowledgements
5a9ab5c
Removed unnecessary logs
057b9d1
Fixed unnecessary change
211ac55
Fixed build failure in KafkaSourceJsonTypeIT
e8455cf
Fix sqs worker integration test failure
740cb4b
Addressed review comments
d3bfac4
Added Progress Check interface and test files
cf6e8f8
Addressed review comments
1b8fad6
Addressed review comments
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 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
This file contains 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
This file contains 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
This file contains 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
This file contains 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 |
---|---|---|
|
@@ -7,6 +7,7 @@ | |
|
||
import org.opensearch.dataprepper.model.event.JacksonEvent; | ||
import org.opensearch.dataprepper.model.event.DefaultEventHandle; | ||
import org.opensearch.dataprepper.metrics.PluginMetrics; | ||
|
||
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; | ||
import org.junit.jupiter.api.BeforeEach; | ||
|
@@ -22,14 +23,14 @@ | |
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.equalTo; | ||
import java.time.Duration; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import java.util.concurrent.Executors; | ||
|
||
@ExtendWith(MockitoExtension.class) | ||
class DefaultAcknowledgementSetManagerTests { | ||
private static final Duration TEST_TIMEOUT = Duration.ofMillis(400); | ||
DefaultAcknowledgementSetManager acknowledgementSetManager; | ||
private ExecutorService callbackExecutor; | ||
private DefaultAcknowledgementSetManager acknowledgementSetManager; | ||
private ScheduledExecutorService callbackExecutor; | ||
|
||
@Mock | ||
JacksonEvent event1; | ||
|
@@ -38,17 +39,24 @@ class DefaultAcknowledgementSetManagerTests { | |
@Mock | ||
JacksonEvent event3; | ||
|
||
DefaultEventHandle eventHandle1; | ||
DefaultEventHandle eventHandle2; | ||
DefaultEventHandle eventHandle3; | ||
Boolean result; | ||
private PluginMetrics pluginMetrics; | ||
private DefaultEventHandle eventHandle1; | ||
private DefaultEventHandle eventHandle2; | ||
private DefaultEventHandle eventHandle3; | ||
private DefaultEventHandle eventHandle4; | ||
private DefaultEventHandle eventHandle5; | ||
private DefaultEventHandle eventHandle6; | ||
Comment on lines
+43
to
+48
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. Nit: Could make this a |
||
private Boolean result; | ||
private double currentRatio; | ||
|
||
@BeforeEach | ||
void setup() { | ||
callbackExecutor = Executors.newFixedThreadPool(2); | ||
currentRatio = 0; | ||
callbackExecutor = Executors.newScheduledThreadPool(2); | ||
event1 = mock(JacksonEvent.class); | ||
eventHandle1 = mock(DefaultEventHandle.class); | ||
lenient().when(event1.getEventHandle()).thenReturn(eventHandle1); | ||
pluginMetrics = mock(PluginMetrics.class); | ||
|
||
event2 = mock(JacksonEvent.class); | ||
eventHandle2 = mock(DefaultEventHandle.class); | ||
|
@@ -76,16 +84,17 @@ void testBasic() { | |
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); | ||
assertThat(result, equalTo(true)); | ||
}); | ||
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); | ||
assertThat(result, equalTo(true)); | ||
} | ||
|
||
@Test | ||
void testExpirations() throws InterruptedException { | ||
acknowledgementSetManager.releaseEventReference(eventHandle2, true); | ||
Thread.sleep(TEST_TIMEOUT.multipliedBy(5).toMillis()); | ||
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); | ||
assertThat(result, equalTo(null)); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(result, equalTo(null)); | ||
}); | ||
} | ||
|
||
@Test | ||
|
@@ -106,7 +115,107 @@ void testMultipleAcknowledgementSets() { | |
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); | ||
assertThat(result, equalTo(true)); | ||
}); | ||
assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); | ||
assertThat(result, equalTo(true)); | ||
} | ||
|
||
@Test | ||
void testWithProgressCheckCallbacks() { | ||
eventHandle3 = mock(DefaultEventHandle.class); | ||
lenient().when(event3.getEventHandle()).thenReturn(eventHandle3); | ||
|
||
eventHandle4 = mock(DefaultEventHandle.class); | ||
JacksonEvent event4 = mock(JacksonEvent.class); | ||
lenient().when(event4.getEventHandle()).thenReturn(eventHandle4); | ||
|
||
eventHandle5 = mock(DefaultEventHandle.class); | ||
JacksonEvent event5 = mock(JacksonEvent.class); | ||
lenient().when(event5.getEventHandle()).thenReturn(eventHandle5); | ||
|
||
eventHandle6 = mock(DefaultEventHandle.class); | ||
JacksonEvent event6 = mock(JacksonEvent.class); | ||
lenient().when(event6.getEventHandle()).thenReturn(eventHandle6); | ||
|
||
AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofMillis(10000)); | ||
acknowledgementSet2.addProgressCheck((ratio) -> {currentRatio = ratio;}, Duration.ofSeconds(1)); | ||
acknowledgementSet2.add(event3); | ||
acknowledgementSet2.add(event4); | ||
acknowledgementSet2.add(event5); | ||
acknowledgementSet2.add(event6); | ||
lenient().when(eventHandle3.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
lenient().when(eventHandle4.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
lenient().when(eventHandle5.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
lenient().when(eventHandle6.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
acknowledgementSet2.complete(); | ||
acknowledgementSetManager.releaseEventReference(eventHandle3, true); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(currentRatio, equalTo(0.75)); | ||
}); | ||
acknowledgementSetManager.releaseEventReference(eventHandle4, true); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(currentRatio, equalTo(0.5)); | ||
}); | ||
acknowledgementSetManager.releaseEventReference(eventHandle5, true); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(currentRatio, equalTo(0.25)); | ||
}); | ||
acknowledgementSetManager.releaseEventReference(eventHandle6, true); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(result, equalTo(true)); | ||
}); | ||
|
||
} | ||
|
||
@Test | ||
void testWithProgressCheckCallbacks_AcksExpire() { | ||
eventHandle3 = mock(DefaultEventHandle.class); | ||
lenient().when(event3.getEventHandle()).thenReturn(eventHandle3); | ||
|
||
eventHandle4 = mock(DefaultEventHandle.class); | ||
JacksonEvent event4 = mock(JacksonEvent.class); | ||
lenient().when(event4.getEventHandle()).thenReturn(eventHandle4); | ||
|
||
eventHandle5 = mock(DefaultEventHandle.class); | ||
JacksonEvent event5 = mock(JacksonEvent.class); | ||
lenient().when(event5.getEventHandle()).thenReturn(eventHandle5); | ||
|
||
eventHandle6 = mock(DefaultEventHandle.class); | ||
JacksonEvent event6 = mock(JacksonEvent.class); | ||
lenient().when(event6.getEventHandle()).thenReturn(eventHandle6); | ||
|
||
AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofSeconds(10)); | ||
acknowledgementSet2.addProgressCheck((ratio) -> {currentRatio = ratio;}, Duration.ofSeconds(1)); | ||
acknowledgementSet2.add(event3); | ||
acknowledgementSet2.add(event4); | ||
acknowledgementSet2.add(event5); | ||
acknowledgementSet2.add(event6); | ||
lenient().when(eventHandle3.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
lenient().when(eventHandle4.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
lenient().when(eventHandle5.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
lenient().when(eventHandle6.getAcknowledgementSet()).thenReturn(acknowledgementSet2); | ||
acknowledgementSet2.complete(); | ||
acknowledgementSetManager.releaseEventReference(eventHandle3, true); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(currentRatio, equalTo(0.75)); | ||
}); | ||
acknowledgementSetManager.releaseEventReference(eventHandle4, true); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(currentRatio, equalTo(0.5)); | ||
}); | ||
acknowledgementSetManager.releaseEventReference(eventHandle5, true); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(currentRatio, equalTo(0.25)); | ||
}); | ||
await().atMost(TEST_TIMEOUT.multipliedBy(5)) | ||
.untilAsserted(() -> { | ||
assertThat(result, equalTo(null)); | ||
}); | ||
|
||
} | ||
|
||
} |
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.
I'm curious why you chose to make this a
Consumer<Double>
. The current code isn't using this ratio presently.A few alternatives:
Runnable
without any input.Consumer<ProgressCheck>
.Then we could add other items if desired such as
Duration getTimeToExpiry()
.