Skip to content
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

[fix][io] Close the kafka source connector got stuck #20698

Merged
merged 7 commits into from
Jun 30, 2023
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions pulsar-io/kafka/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,12 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.awaitility</groupId>
<artifactId>awaitility</artifactId>
<scope>test</scope>
</dependency>

</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.pulsar.io.kafka;

import io.jsonwebtoken.io.Encoders;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.time.Duration;
import java.util.Collections;
import java.util.HashMap;
Expand All @@ -28,6 +29,8 @@
import java.util.Properties;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
Expand Down Expand Up @@ -64,6 +67,8 @@ public abstract class KafkaAbstractSource<V> extends PushSource<V> {
private volatile boolean running = false;
private KafkaSourceConfig kafkaSourceConfig;
private Thread runnerThread;
private final static Executor EXECUTOR = Executors.newSingleThreadExecutor(
new DefaultThreadFactory("Kafka Source Close Task Thread"));

@Override
public void open(Map<String, Object> config, SourceContext sourceContext) throws Exception {
Expand Down Expand Up @@ -190,12 +195,14 @@ public void start() {
});
runnerThread.setUncaughtExceptionHandler(
(t, e) -> {
LOG.error("[{}] Error while consuming records", t.getName(), e);
try {
this.close();
} catch (InterruptedException ex) {
// The interrupted exception is thrown by the runnerThread itself. Ignore it.
}
EXECUTOR.execute(() -> {
LOG.error("[{}] Error while consuming records", t.getName(), e);
try {
this.close();
} catch (Exception ex) {
LOG.error("[{}] Close kafka source error", t.getName(), e);
}
});
});
runnerThread.setName("Kafka Source Thread");
runnerThread.start();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@


import com.google.common.collect.ImmutableMap;
import io.netty.handler.codec.spdy.DefaultSpdyGoAwayFrame;
import java.util.Collection;
import java.util.Collections;
import java.lang.reflect.Field;
Expand All @@ -31,6 +32,7 @@
import org.apache.pulsar.io.core.SourceContext;
import org.apache.pulsar.io.kafka.KafkaAbstractSource;
import org.apache.pulsar.io.kafka.KafkaSourceConfig;
import org.awaitility.Awaitility;
import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.Test;
Expand Down Expand Up @@ -173,7 +175,10 @@ public final void closeConnectorWhenUnexpectedExceptionThrownTest() throws Excep
Field runningField = KafkaAbstractSource.class.getDeclaredField("running");
runningField.setAccessible(true);

Assert.assertFalse((boolean) runningField.get(source));
Awaitility.await().untilAsserted(() -> {
Assert.assertFalse((boolean) runningField.get(source));
Assert.assertNull(consumerField.get(source));
});
}

private File getFile(String name) {
Expand Down