-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-14780: Fix flaky test 'testSecondaryRefreshAfterElapsedDelay' #14078
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -25,11 +25,21 @@ | |
| import static org.mockito.Mockito.times; | ||
| import static org.mockito.Mockito.verify; | ||
|
|
||
| import java.util.Arrays; | ||
| import java.util.Collection; | ||
| import java.util.Collections; | ||
| import java.util.Collection; | ||
| import java.util.List; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.AbstractMap; | ||
| import java.util.Map; | ||
| import java.util.TreeMap; | ||
| import java.util.concurrent.Callable; | ||
| import java.util.concurrent.ScheduledExecutorService; | ||
| import java.util.concurrent.ScheduledFuture; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| import org.apache.kafka.common.KafkaFuture; | ||
| import org.apache.kafka.common.internals.KafkaFutureImpl; | ||
| import org.apache.kafka.common.utils.MockTime; | ||
| import org.apache.kafka.common.utils.Time; | ||
| import org.jose4j.http.SimpleResponse; | ||
|
|
@@ -122,14 +132,36 @@ public void testLongKey() throws Exception { | |
| @Test | ||
| public void testSecondaryRefreshAfterElapsedDelay() throws Exception { | ||
| String keyId = "abc123"; | ||
| Time time = MockTime.SYSTEM; // Unfortunately, we can't mock time here because the | ||
| // scheduled executor doesn't respect it. | ||
| MockTime time = new MockTime(); | ||
| HttpsJwks httpsJwks = spyHttpsJwks(); | ||
|
|
||
| try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks)) { | ||
| MockExecutorService mockExecutorService = new MockExecutorService(time); | ||
| ScheduledExecutorService executorService = Mockito.mock(ScheduledExecutorService.class); | ||
| Mockito.doAnswer(invocation -> { | ||
| Runnable command = invocation.getArgument(0, Runnable.class); | ||
| long delay = invocation.getArgument(1, Long.class); | ||
| TimeUnit unit = invocation.getArgument(2, TimeUnit.class); | ||
| return mockExecutorService.schedule(() -> { | ||
| command.run(); | ||
| return null; | ||
| }, unit.toMillis(delay), null); | ||
| }).when(executorService).schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class)); | ||
| Mockito.doAnswer(invocation -> { | ||
| Runnable command = invocation.getArgument(0, Runnable.class); | ||
| long initialDelay = invocation.getArgument(1, Long.class); | ||
| long period = invocation.getArgument(2, Long.class); | ||
| TimeUnit unit = invocation.getArgument(3, TimeUnit.class); | ||
| return mockExecutorService.schedule(() -> { | ||
| command.run(); | ||
| return null; | ||
| }, unit.toMillis(initialDelay), period); | ||
| }).when(executorService).scheduleAtFixedRate(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.anyLong(), Mockito.any(TimeUnit.class)); | ||
|
|
||
| try (RefreshingHttpsJwks refreshingHttpsJwks = getRefreshingHttpsJwks(time, httpsJwks, executorService)) { | ||
| refreshingHttpsJwks.init(); | ||
| // We refresh once at the initialization time from getJsonWebKeys. | ||
| verify(httpsJwks, times(1)).refresh(); | ||
| assertTrue(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); | ||
| verify(httpsJwks, times(2)).refresh(); | ||
| time.sleep(REFRESH_MS + 1); | ||
| verify(httpsJwks, times(3)).refresh(); | ||
| assertFalse(refreshingHttpsJwks.maybeExpediteRefresh(keyId)); | ||
|
|
@@ -153,6 +185,10 @@ private RefreshingHttpsJwks getRefreshingHttpsJwks(final Time time, final HttpsJ | |
| return new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS, RETRY_BACKOFF_MS, RETRY_BACKOFF_MAX_MS); | ||
| } | ||
|
|
||
| private RefreshingHttpsJwks getRefreshingHttpsJwks(final Time time, final HttpsJwks httpsJwks, final ScheduledExecutorService executorService) { | ||
| return new RefreshingHttpsJwks(time, httpsJwks, REFRESH_MS, RETRY_BACKOFF_MS, RETRY_BACKOFF_MAX_MS, executorService); | ||
| } | ||
|
|
||
| /** | ||
| * We *spy* (not *mock*) the {@link HttpsJwks} instance because we want to have it | ||
| * _partially mocked_ to determine if it's calling its internal refresh method. We want to | ||
|
|
@@ -195,4 +231,82 @@ public String getBody() { | |
| return Mockito.spy(httpsJwks); | ||
| } | ||
|
|
||
| } | ||
| /** | ||
| * A mock ScheduledExecutorService just for the test. Note that this is not a generally reusable mock as it does not | ||
| * implement some interfaces like scheduleWithFixedDelay, etc. And it does not return ScheduledFuture correctly. | ||
| */ | ||
| private class MockExecutorService implements MockTime.Listener { | ||
| private final MockTime time; | ||
|
|
||
| private final TreeMap<Long, List<AbstractMap.SimpleEntry<Long, KafkaFutureImpl<Long>>>> waiters = new TreeMap<>(); | ||
|
|
||
| public MockExecutorService(MockTime time) { | ||
| this.time = time; | ||
| time.addListener(this); | ||
| } | ||
|
|
||
| /** | ||
| * The actual execution and rescheduling logic. Check all internal tasks to see if any one reaches its next | ||
| * execution point, call it and optionally reschedule it if it has a specified period. | ||
| */ | ||
| @Override | ||
| public synchronized void onTimeUpdated() { | ||
|
Member
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. Could you add some java doc on what we're trying to do in this method? Same as below. Thanks.
Contributor
Author
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. Done |
||
| long timeMs = time.milliseconds(); | ||
| while (true) { | ||
| Map.Entry<Long, List<AbstractMap.SimpleEntry<Long, KafkaFutureImpl<Long>>>> entry = waiters.firstEntry(); | ||
| if ((entry == null) || (entry.getKey() > timeMs)) { | ||
| break; | ||
| } | ||
| for (AbstractMap.SimpleEntry<Long, KafkaFutureImpl<Long>> pair : entry.getValue()) { | ||
| pair.getValue().complete(timeMs); | ||
| if (pair.getKey() != null) { | ||
| addWaiter(entry.getKey() + pair.getKey(), pair.getKey(), pair.getValue()); | ||
| } | ||
| } | ||
| waiters.remove(entry.getKey()); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Add a task with `delayMs` and optional period to the internal waiter. | ||
| * When `delayMs` < 0, we immediately complete the waiter. Otherwise, we add the task metadata to the waiter and | ||
| * onTimeUpdated will take care of execute and reschedule it when it reaches its scheduled timestamp. | ||
| * | ||
| * @param delayMs Delay time in ms. | ||
| * @param period Scheduling period, null means no periodic. | ||
| * @param waiter A wrapper over a callable function. | ||
| */ | ||
| private synchronized void addWaiter(long delayMs, Long period, KafkaFutureImpl<Long> waiter) { | ||
| long timeMs = time.milliseconds(); | ||
| if (delayMs <= 0) { | ||
| waiter.complete(timeMs); | ||
| } else { | ||
| long triggerTimeMs = timeMs + delayMs; | ||
| List<AbstractMap.SimpleEntry<Long, KafkaFutureImpl<Long>>> futures = | ||
| waiters.computeIfAbsent(triggerTimeMs, k -> new ArrayList<>()); | ||
| futures.add(new AbstractMap.SimpleEntry<>(period, waiter)); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Internal utility function for periodic or one time refreshes. | ||
| * | ||
| * @param period null indicates one time refresh, otherwise it is periodic. | ||
| */ | ||
| public <T> ScheduledFuture<T> schedule(final Callable<T> callable, long delayMs, Long period) { | ||
|
|
||
| KafkaFutureImpl<Long> waiter = new KafkaFutureImpl<>(); | ||
| waiter.thenApply((KafkaFuture.BaseFunction<Long, Void>) now -> { | ||
| try { | ||
| callable.call(); | ||
| } catch (Throwable e) { | ||
| e.printStackTrace(); | ||
| } | ||
|
Member
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. Should we log anything here for future troubleshooting?
Contributor
Author
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. This is a test-only class so I think it should be fine? We should make sure scheduled thing does not have exceptions?
Member
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. OK, but we sometimes do It will be helpful when this test someday become flaky. Could we add it? |
||
| return null; | ||
| }); | ||
| addWaiter(delayMs, period, waiter); | ||
| return 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 can see it is basically identical to the
MockScheduler. Could you explain why can't we useMockSchedulerdirectly here?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.
Yes it is very similar but I have to create my own mock for 2 reasons: