mirror of https://github.com/apache/kafka.git
KAFKA-19450: ShareConsumerPerformance does not handle exceptions from consumeMessagesForSingleShareConsumer (#20126)
### About Within `ShareConsumerPerformance.java`, all the share consumers run with within an executorService object and when we perform `executorService.submit()`, we do not store this future and exception would be recovered only when we do a future.get() in this case. I believe this is a shortcoming in `ShareConsumerPerformance.java` which needs to be improved. Reviewers: Andrew Schofield <aschofield@confluent.io>
This commit is contained in:
parent
36b9bb94f1
commit
e489682c45
|
@ -43,9 +43,12 @@ import java.util.Map;
|
|||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import joptsimple.OptionException;
|
||||
|
@ -108,7 +111,7 @@ public class ShareConsumerPerformance {
|
|||
ShareConsumerPerfOptions options,
|
||||
AtomicLong totalMessagesRead,
|
||||
AtomicLong totalBytesRead,
|
||||
long startMs) {
|
||||
long startMs) throws ExecutionException, InterruptedException, TimeoutException {
|
||||
long numMessages = options.numMessages();
|
||||
long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
|
||||
shareConsumers.forEach(shareConsumer -> shareConsumer.subscribe(options.topic()));
|
||||
|
@ -120,17 +123,18 @@ public class ShareConsumerPerformance {
|
|||
|
||||
|
||||
ExecutorService executorService = Executors.newFixedThreadPool(shareConsumers.size());
|
||||
List<Future<?>> futures = new ArrayList<>();
|
||||
for (int i = 0; i < shareConsumers.size(); i++) {
|
||||
final int index = i;
|
||||
ShareConsumerConsumption shareConsumerConsumption = new ShareConsumerConsumption(0, 0);
|
||||
executorService.submit(() -> {
|
||||
futures.add(executorService.submit(() -> {
|
||||
try {
|
||||
consumeMessagesForSingleShareConsumer(shareConsumers.get(index), messagesRead, bytesRead, options,
|
||||
shareConsumerConsumption, index + 1);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
}));
|
||||
shareConsumersConsumptionDetails.add(shareConsumerConsumption);
|
||||
}
|
||||
LOG.debug("Shutting down of thread pool is started");
|
||||
|
@ -153,6 +157,9 @@ public class ShareConsumerPerformance {
|
|||
// Preserve interrupt status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
for (Future<?> future : futures) {
|
||||
future.get();
|
||||
}
|
||||
|
||||
if (options.showShareConsumerStats()) {
|
||||
long endMs = System.currentTimeMillis();
|
||||
|
|
Loading…
Reference in New Issue