Skip to content

Commit efd7852

Browse files
authored
KAFKA-19124: Follow up on code improvements (#19453)
Improves a variable name and handling of an Optional. Reviewers: Bill Bejeck <bill@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang <payang@apache.org>
1 parent 6fe1598 commit efd7852

File tree

1 file changed

+8
-13
lines changed

1 file changed

+8
-13
lines changed

clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java

+8-13
Original file line numberDiff line numberDiff line change
@@ -186,7 +186,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
186186
*/
187187
private class BackgroundEventProcessor implements EventProcessor<BackgroundEvent> {
188188

189-
private Optional<StreamsRebalanceListener> streamsGroupRebalanceCallbacks = Optional.empty();
189+
private Optional<StreamsRebalanceListener> streamsRebalanceListener = Optional.empty();
190190
private final Optional<StreamsRebalanceData> streamsRebalanceData;
191191

192192
public BackgroundEventProcessor() {
@@ -202,7 +202,7 @@ private void setStreamsRebalanceListener(final StreamsRebalanceListener streamsR
202202
throw new IllegalStateException("Background event processor was not created to be used with Streams " +
203203
"rebalance protocol events");
204204
}
205-
this.streamsGroupRebalanceCallbacks = Optional.of(streamsRebalanceListener);
205+
this.streamsRebalanceListener = Optional.of(streamsRebalanceListener);
206206
}
207207

208208
@Override
@@ -277,20 +277,15 @@ private void processStreamsOnAllTasksLostCallbackNeededEvent(final StreamsOnAllT
277277

278278
private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> activeTasksToRevoke,
279279
final CompletableFuture<Void> future) {
280-
final Optional<KafkaException> error;
281-
final Optional<Exception> exceptionFromCallback = streamsGroupRebalanceCallbacks().onTasksRevoked(activeTasksToRevoke);
282-
if (exceptionFromCallback.isPresent()) {
283-
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task revocation callback throws an error"));
284-
} else {
285-
error = Optional.empty();
286-
}
280+
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke);
281+
final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws an error"));
287282
return new StreamsOnTasksRevokedCallbackCompletedEvent(future, error);
288283
}
289284

290285
private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment,
291286
final CompletableFuture<Void> future) {
292287
final Optional<KafkaException> error;
293-
final Optional<Exception> exceptionFromCallback = streamsGroupRebalanceCallbacks().onTasksAssigned(assignment);
288+
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onTasksAssigned(assignment);
294289
if (exceptionFromCallback.isPresent()) {
295290
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task assignment callback throws an error"));
296291
} else {
@@ -302,7 +297,7 @@ private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallba
302297

303298
private StreamsOnAllTasksLostCallbackCompletedEvent invokeOnAllTasksLostCallback(final CompletableFuture<Void> future) {
304299
final Optional<KafkaException> error;
305-
final Optional<Exception> exceptionFromCallback = streamsGroupRebalanceCallbacks().onAllTasksLost();
300+
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onAllTasksLost();
306301
if (exceptionFromCallback.isPresent()) {
307302
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "All tasks lost callback throws an error"));
308303
} else {
@@ -318,8 +313,8 @@ private StreamsRebalanceData streamsRebalanceData() {
318313
"rebalance protocol events"));
319314
}
320315

321-
private StreamsRebalanceListener streamsGroupRebalanceCallbacks() {
322-
return streamsGroupRebalanceCallbacks.orElseThrow(
316+
private StreamsRebalanceListener streamsRebalanceListener() {
317+
return streamsRebalanceListener.orElseThrow(
323318
() -> new IllegalStateException("Background event processor was not created to be used with Streams " +
324319
"rebalance protocol events"));
325320
}

0 commit comments

Comments
 (0)