-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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
KAFKA-15942: Implement ConsumerInterceptors in the async consumer #15000
Changes from 1 commit
db405f1
8ca984f
bc2a8da
e103568
8ec95f9
b6be0b9
4985460
7e405d1
73fd928
31cf64c
68740cd
4434f4b
c6d07dd
47b21e1
4325279
fd687dd
2d6f231
4b1c1c4
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 | ||
---|---|---|---|---|
|
@@ -33,6 +33,7 @@ | |||
*/ | ||||
public class OffsetCommitCallbackInvoker { | ||||
private final ConsumerInterceptors<?, ?> interceptors; | ||||
private boolean hasFencedException = false; | ||||
|
||||
OffsetCommitCallbackInvoker(ConsumerInterceptors<?, ?> interceptors) { | ||||
this.interceptors = interceptors; | ||||
|
@@ -57,23 +58,22 @@ public void submitUserCallback(final OffsetCommitCallback callback, | |||
callbackQueue.add(new OffsetCommitCallbackTask(callback, offsets, exception)); | ||||
} | ||||
|
||||
/** | ||||
* @return true if an offset commit was fenced. | ||||
*/ | ||||
public boolean executeCallbacks() { | ||||
boolean isFenced = false; | ||||
public void executeCallbacks() { | ||||
while (!callbackQueue.isEmpty()) { | ||||
OffsetCommitCallbackTask task = callbackQueue.poll(); | ||||
if (task != null) { | ||||
|
||||
if (task.exception instanceof FencedInstanceIdException) | ||||
isFenced = true; | ||||
hasFencedException = true; | ||||
|
||||
task.callback.onComplete(task.offsets, task.exception); | ||||
|
||||
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:
Suggested change
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 |
||||
} | ||||
} | ||||
return isFenced; | ||||
} | ||||
|
||||
public boolean hasFencedException() { | ||||
return hasFencedException; | ||||
} | ||||
|
||||
private static class OffsetCommitCallbackTask { | ||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -849,6 +849,22 @@ public void testWakeupCommitted() { | |
assertNull(consumer.wakeupTrigger().getPendingTask()); | ||
} | ||
|
||
@Test | ||
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. what's the interceptor behavior on close? if we have inflight commits before closing the consumer, should the interceptors be invoked? can we add tests around that? 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. Added a test. Interceptors are called. |
||
public void testInterceptorAutoCommitOnClose() { | ||
Properties props = requiredConsumerPropertiesAndGroupId("test-id"); | ||
props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); | ||
props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); | ||
|
||
consumer = newConsumer(props); | ||
assertEquals(1, MockConsumerInterceptor.INIT_COUNT.get()); | ||
completeCommitApplicationEventSuccessfully(); | ||
|
||
consumer.close(Duration.ZERO); | ||
|
||
assertEquals(1, MockConsumerInterceptor.ON_COMMIT_COUNT.get()); | ||
assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get()); | ||
} | ||
|
||
@Test | ||
public void testInterceptorCommitSync() { | ||
Properties props = requiredConsumerPropertiesAndGroupId("test-id"); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1336,8 +1336,9 @@ class PlaintextConsumerTest extends BaseConsumerTest { | |
MockProducerInterceptor.resetCounters() | ||
} | ||
|
||
// This is disabled for the the consumer group until KAFKA-16155 is resolved. | ||
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 for reporting this. |
||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) | ||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) | ||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) | ||
def testAutoCommitIntercept(quorum: String, groupProtocol: String): Unit = { | ||
val topic2 = "topic2" | ||
createTopic(topic2, 2, brokerCount) | ||
|
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.
looking back at the implementation - now i think it is rather unnecessary to have these finals in the parameters. I wonder if you think we should clean them up in the future...
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 is very mixed across the clients codebase. Sometimes you put the final sometimes you don't. In the streams module there is a strict rule to do. Not sure, but as long as there is no guideline around this, and we are not completely repulsed by it, I'd suggest to just stick with whatever the existing code is doing for consistency to not mess up git blame too much.