lianetm commented on code in PR #17699:
URL: https://github.com/apache/kafka/pull/17699#discussion_r1852757098
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1170,25 +1174,11 @@ private Map<TopicPartition, Long>
beginningOrEndOffset(Collection<TopicPartition
public OptionalLong currentLag(TopicPartition topicPartition) {
acquireAndEnsureOpen();
try {
- final Long lag = subscriptions.partitionLag(topicPartition,
isolationLevel);
-
- // if the log end offset is not known and hence cannot return lag
and there is
- // no in-flight list offset requested yet,
- // issue a list offset request for that partition so that next time
- // we may get the answer; we do not need to wait for the return
value
- // since we would not try to poll the network client synchronously
Review Comment:
this part of the comment about polling the network client seems a bit
confusing here now, doesn't it? Is it maybe enough until the ";" (it's clear
there that we're issuing a request if needed but not waiting)
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -515,6 +531,89 @@ private void process(final
ShareAcknowledgementCommitCallbackRegistrationEvent e
manager.setAcknowledgementCommitCallbackRegistered(event.isCallbackRegistered());
}
+ private void process(final SeekUnvalidatedEvent event) {
+ try {
+ event.offsetEpoch().ifPresent(epoch ->
metadata.updateLastSeenEpochIfNewer(event.partition(), epoch));
+ SubscriptionState.FetchPosition newPosition = new
SubscriptionState.FetchPosition(
+ event.offset(),
+ event.offsetEpoch(),
+ metadata.currentLeader(event.partition())
+ );
+ subscriptions.seekUnvalidated(event.partition(), newPosition);
+ event.future().complete(null);
+ } catch (Exception e) {
+ event.future().completeExceptionally(e);
+ }
+ }
+
+ private void process(final PausePartitionsEvent event) {
+ try {
+ Collection<TopicPartition> partitions = event.partitions();
+ log.debug("Pausing partitions {}", partitions);
+
+ for (TopicPartition partition : partitions) {
+ subscriptions.pause(partition);
+ }
+
+ event.future().complete(null);
+ } catch (Exception e) {
+ event.future().completeExceptionally(e);
+ }
+ }
+
+ private void process(final ResumePartitionsEvent event) {
+ try {
+ Collection<TopicPartition> partitions = event.partitions();
+ log.debug("Resuming partitions {}", partitions);
+
+ for (TopicPartition partition : partitions) {
+ subscriptions.resume(partition);
+ }
+
+ event.future().complete(null);
+ } catch (Exception e) {
+ event.future().completeExceptionally(e);
+ }
+ }
+
+ private void process(final CurrentLagEvent event) {
Review Comment:
we have no unit test coverage for this now, could we add some? (same for
pause/resume)
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1045,18 +1052,15 @@ public void pause(Collection<TopicPartition>
partitions) {
public void resume(Collection<TopicPartition> partitions) {
acquireAndEnsureOpen();
try {
- log.debug("Resuming partitions {}", partitions);
- for (TopicPartition partition : partitions) {
- subscriptions.resume(partition);
- }
+ applicationEventHandler.addAndGet(new
ResumePartitionsEvent(partitions, defaultApiTimeoutDeadlineMs()));
Review Comment:
ditto
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1032,10 +1042,7 @@ public Set<TopicPartition> paused() {
public void pause(Collection<TopicPartition> partitions) {
acquireAndEnsureOpen();
try {
- log.debug("Pausing partitions {}", partitions);
- for (TopicPartition partition : partitions) {
- subscriptions.pause(partition);
- }
+ applicationEventHandler.addAndGet(new
PausePartitionsEvent(partitions, defaultApiTimeoutDeadlineMs()));
Review Comment:
what about validating null or empty here at the API level to avoid
generating and waiting on a background event in those cases? We should simply
throw NPE on null and no-op on empty to keep the current/classic behaviour I
believe.
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java:
##########
@@ -33,7 +33,7 @@ public enum Type {
LIST_OFFSETS, CHECK_AND_UPDATE_POSITIONS, RESET_OFFSET,
TOPIC_METADATA, ALL_TOPICS_METADATA,
TOPIC_SUBSCRIPTION_CHANGE, TOPIC_PATTERN_SUBSCRIPTION_CHANGE,
UPDATE_SUBSCRIPTION_METADATA,
UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED,
- COMMIT_ON_CLOSE, CREATE_FETCH_REQUESTS, LEAVE_GROUP_ON_CLOSE,
+ COMMIT_ON_CLOSE, CREATE_FETCH_REQUESTS, LEAVE_GROUP_ON_CLOSE,
PAUSE_PARTITIONS, RESUME_PARTITIONS, CURRENT_LAG,
Review Comment:
nit: should we add the 3 new ones that somehow relate in a new line?
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -546,21 +645,6 @@ protected ApplicationEventProcessor create() {
};
}
- private void process(final SeekUnvalidatedEvent event) {
Review Comment:
no changes in this seekUnvalidated right? (just moving it up next to all
other process calls)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]