mirror of https://github.com/apache/kafka.git
KAFKA-19271: allow intercepting internal method call (#19832)
CI / build (push) Waiting to run
Details
CI / build (push) Waiting to run
Details
To allow intercepting the internal subscribe call to the async-consumer, we need to extend ConsumerWrapper interface accordingly, instead of returning the wrapped async-consumer back to the KS runtime. Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
This commit is contained in:
parent
948a91dfdf
commit
0adc6fa3e1
|
@ -27,6 +27,7 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback;
|
||||||
import org.apache.kafka.clients.consumer.SubscriptionPattern;
|
import org.apache.kafka.clients.consumer.SubscriptionPattern;
|
||||||
import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer;
|
import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer;
|
||||||
import org.apache.kafka.clients.consumer.internals.StreamsRebalanceData;
|
import org.apache.kafka.clients.consumer.internals.StreamsRebalanceData;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.StreamsRebalanceListener;
|
||||||
import org.apache.kafka.common.Metric;
|
import org.apache.kafka.common.Metric;
|
||||||
import org.apache.kafka.common.MetricName;
|
import org.apache.kafka.common.MetricName;
|
||||||
import org.apache.kafka.common.PartitionInfo;
|
import org.apache.kafka.common.PartitionInfo;
|
||||||
|
@ -54,10 +55,6 @@ public abstract class ConsumerWrapper implements Consumer<byte[], byte[]> {
|
||||||
this.delegate = delegate;
|
this.delegate = delegate;
|
||||||
}
|
}
|
||||||
|
|
||||||
public AsyncKafkaConsumer<byte[], byte[]> consumer() {
|
|
||||||
return delegate;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<TopicPartition> assignment() {
|
public Set<TopicPartition> assignment() {
|
||||||
return delegate.assignment();
|
return delegate.assignment();
|
||||||
|
@ -78,6 +75,10 @@ public abstract class ConsumerWrapper implements Consumer<byte[], byte[]> {
|
||||||
delegate.subscribe(topics, callback);
|
delegate.subscribe(topics, callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void subscribe(final Collection<String> topics, final StreamsRebalanceListener streamsRebalanceListener) {
|
||||||
|
delegate.subscribe(topics, streamsRebalanceListener);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void assign(final Collection<TopicPartition> partitions) {
|
public void assign(final Collection<TopicPartition> partitions) {
|
||||||
delegate.assign(partitions);
|
delegate.assign(partitions);
|
||||||
|
|
|
@ -1138,19 +1138,29 @@ public class StreamThread extends Thread implements ProcessingThread {
|
||||||
mainConsumer.subscribe(topologyMetadata.sourceTopicPattern(), rebalanceListener);
|
mainConsumer.subscribe(topologyMetadata.sourceTopicPattern(), rebalanceListener);
|
||||||
} else {
|
} else {
|
||||||
if (streamsRebalanceData.isPresent()) {
|
if (streamsRebalanceData.isPresent()) {
|
||||||
final AsyncKafkaConsumer<byte[], byte[]> consumer = mainConsumer instanceof ConsumerWrapper
|
if (mainConsumer instanceof ConsumerWrapper) {
|
||||||
? ((ConsumerWrapper) mainConsumer).consumer()
|
((ConsumerWrapper) mainConsumer).subscribe(
|
||||||
: (AsyncKafkaConsumer<byte[], byte[]>) mainConsumer;
|
topologyMetadata.allFullSourceTopicNames(),
|
||||||
consumer.subscribe(
|
new DefaultStreamsRebalanceListener(
|
||||||
topologyMetadata.allFullSourceTopicNames(),
|
log,
|
||||||
new DefaultStreamsRebalanceListener(
|
time,
|
||||||
log,
|
streamsRebalanceData.get(),
|
||||||
time,
|
this,
|
||||||
streamsRebalanceData.get(),
|
taskManager
|
||||||
this,
|
)
|
||||||
taskManager
|
);
|
||||||
)
|
} else {
|
||||||
);
|
((AsyncKafkaConsumer<byte[], byte[]>) mainConsumer).subscribe(
|
||||||
|
topologyMetadata.allFullSourceTopicNames(),
|
||||||
|
new DefaultStreamsRebalanceListener(
|
||||||
|
log,
|
||||||
|
time,
|
||||||
|
streamsRebalanceData.get(),
|
||||||
|
this,
|
||||||
|
taskManager
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
mainConsumer.subscribe(topologyMetadata.allFullSourceTopicNames(), rebalanceListener);
|
mainConsumer.subscribe(topologyMetadata.allFullSourceTopicNames(), rebalanceListener);
|
||||||
}
|
}
|
||||||
|
|
|
@ -69,7 +69,6 @@ import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler;
|
||||||
import org.apache.kafka.streams.errors.StreamsException;
|
import org.apache.kafka.streams.errors.StreamsException;
|
||||||
import org.apache.kafka.streams.errors.TaskCorruptedException;
|
import org.apache.kafka.streams.errors.TaskCorruptedException;
|
||||||
import org.apache.kafka.streams.errors.TaskMigratedException;
|
import org.apache.kafka.streams.errors.TaskMigratedException;
|
||||||
import org.apache.kafka.streams.internals.ConsumerWrapper;
|
|
||||||
import org.apache.kafka.streams.kstream.Consumed;
|
import org.apache.kafka.streams.kstream.Consumed;
|
||||||
import org.apache.kafka.streams.kstream.Materialized;
|
import org.apache.kafka.streams.kstream.Materialized;
|
||||||
import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
|
import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
|
||||||
|
@ -3952,38 +3951,6 @@ public class StreamThreadTest {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
|
||||||
@MethodSource("data")
|
|
||||||
public void shouldWrapMainConsumerFromClassConfig(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
|
||||||
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
|
||||||
streamsConfigProps.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, "streams");
|
|
||||||
streamsConfigProps.put(InternalConfig.INTERNAL_CONSUMER_WRAPPER, TestWrapper.class);
|
|
||||||
|
|
||||||
thread = createStreamThread("clientId", new StreamsConfig(streamsConfigProps));
|
|
||||||
|
|
||||||
assertInstanceOf(
|
|
||||||
AsyncKafkaConsumer.class,
|
|
||||||
assertInstanceOf(TestWrapper.class, thread.mainConsumer()).consumer()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest
|
|
||||||
@MethodSource("data")
|
|
||||||
public void shouldWrapMainConsumerFromStringConfig(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
|
|
||||||
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
|
|
||||||
streamsConfigProps.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, "streams");
|
|
||||||
streamsConfigProps.put(InternalConfig.INTERNAL_CONSUMER_WRAPPER, TestWrapper.class.getName());
|
|
||||||
|
|
||||||
thread = createStreamThread("clientId", new StreamsConfig(streamsConfigProps));
|
|
||||||
|
|
||||||
assertInstanceOf(
|
|
||||||
AsyncKafkaConsumer.class,
|
|
||||||
assertInstanceOf(TestWrapper.class, thread.mainConsumer()).consumer()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static final class TestWrapper extends ConsumerWrapper { }
|
|
||||||
|
|
||||||
private StreamThread setUpThread(final Properties streamsConfigProps) {
|
private StreamThread setUpThread(final Properties streamsConfigProps) {
|
||||||
final StreamsConfig config = new StreamsConfig(streamsConfigProps);
|
final StreamsConfig config = new StreamsConfig(streamsConfigProps);
|
||||||
final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class);
|
final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class);
|
||||||
|
|
Loading…
Reference in New Issue