[
https://issues.apache.org/jira/browse/GOBBLIN-2177?focusedWorklogId=948188&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-948188
]
ASF GitHub Bot logged work on GOBBLIN-2177:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 13/Dec/24 07:59
Start Date: 13/Dec/24 07:59
Worklog Time Spent: 10m
Work Description: phet commented on code in PR #4080:
URL: https://github.com/apache/gobblin/pull/4080#discussion_r1883504102
##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/kafka/HighLevelConsumer.java:
##########
@@ -340,18 +340,29 @@ public void run() {
while (true) {
record = queue.take();
messagesRead.inc();
- HighLevelConsumer.this.processMessage((DecodeableKafkaRecord)record);
- recordsProcessed.incrementAndGet();
+ try {
+ HighLevelConsumer.this.processMessage((DecodeableKafkaRecord)
record);
+ recordsProcessed.incrementAndGet();
+ }
+ catch (Exception e) {
+ // Rethrow exception in case auto commit is disabled
+ if (!HighLevelConsumer.this.enableAutoCommit) {
+ throw e;
+ }
+ // Continue with processing next records in case auto commit is
enabled
+ log.error("Encountered exception while processing record. Record:
{} Exception: {}", record, e);
+ }
- if(!HighLevelConsumer.this.enableAutoCommit) {
- KafkaPartition partition = new
KafkaPartition.Builder().withId(record.getPartition()).withTopicName(HighLevelConsumer.this.topic).build();
+ if (!HighLevelConsumer.this.enableAutoCommit) {
+ KafkaPartition partition =
+ new
KafkaPartition.Builder().withId(record.getPartition()).withTopicName(HighLevelConsumer.this.topic)
+ .build();
// Committed offset should always be the offset of the next record
to be read (hence +1)
partitionOffsetsToCommit.put(partition, record.getOffset() + 1);
}
}
- } catch (InterruptedException e) {
+ } catch(InterruptedException e){
Review Comment:
needs whitespace
##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/kafka/HighLevelConsumer.java:
##########
@@ -340,18 +340,29 @@ public void run() {
while (true) {
record = queue.take();
messagesRead.inc();
- HighLevelConsumer.this.processMessage((DecodeableKafkaRecord)record);
- recordsProcessed.incrementAndGet();
+ try {
+ HighLevelConsumer.this.processMessage((DecodeableKafkaRecord)
record);
+ recordsProcessed.incrementAndGet();
+ }
+ catch (Exception e) {
+ // Rethrow exception in case auto commit is disabled
+ if (!HighLevelConsumer.this.enableAutoCommit) {
+ throw e;
+ }
+ // Continue with processing next records in case auto commit is
enabled
+ log.error("Encountered exception while processing record. Record:
{} Exception: {}", record, e);
Review Comment:
might be worth adding an external metric here so we could establish an
alarm. we don't want to continue churning through a long sequence of errors
only to be dropping messages w/ nobody realizing
Issue Time Tracking
-------------------
Worklog Id: (was: 948188)
Time Spent: 2.5h (was: 2h 20m)
> Avoid stopping Kafka HighLevelConsumer - QueueProcessor on
> non-InterruptedExceptions
> ------------------------------------------------------------------------------------
>
> Key: GOBBLIN-2177
> URL: https://issues.apache.org/jira/browse/GOBBLIN-2177
> Project: Apache Gobblin
> Issue Type: Bug
> Reporter: Abhishek Jain
> Priority: Major
> Time Spent: 2.5h
> Remaining Estimate: 0h
>
> The QueueProcessor within HighLevelConsumer contains an infinite while loop
> that is enclosed in a try-catch block. When any exception is encountered,
> this loop breaks, which halts the processing of any consumed messages until
> the service is restarted.
> We should not break this infinite loop on all exceptions; rather, we should
> break it only on InterruptedException, which truly means the QueueProcessor
> should stop processing.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)