Skip to content

KAFKA-19271: allow intercepting internal method call #19832

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

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.kafka.clients.consumer.SubscriptionPattern;
import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer;
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.MetricName;
import org.apache.kafka.common.PartitionInfo;
Expand Down Expand Up @@ -54,10 +55,6 @@ public void wrapConsumer(
this.delegate = delegate;
}

public AsyncKafkaConsumer<byte[], byte[]> consumer() {
return delegate;
}

@Override
public Set<TopicPartition> assignment() {
return delegate.assignment();
Expand All @@ -78,6 +75,10 @@ public void subscribe(final Collection<String> topics, final ConsumerRebalanceLi
delegate.subscribe(topics, callback);
}

public void subscribe(final Collection<String> topics, final StreamsRebalanceListener streamsRebalanceListener) {
delegate.subscribe(topics, streamsRebalanceListener);
}

@Override
public void assign(final Collection<TopicPartition> partitions) {
delegate.assign(partitions);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1137,19 +1137,29 @@ private void subscribeConsumer() {
mainConsumer.subscribe(topologyMetadata.sourceTopicPattern(), rebalanceListener);
} else {
if (streamsRebalanceData.isPresent()) {
final AsyncKafkaConsumer<byte[], byte[]> consumer = mainConsumer instanceof ConsumerWrapper
? ((ConsumerWrapper) mainConsumer).consumer()
: (AsyncKafkaConsumer<byte[], byte[]>) mainConsumer;
consumer.subscribe(
topologyMetadata.allFullSourceTopicNames(),
new DefaultStreamsRebalanceListener(
log,
time,
streamsRebalanceData.get(),
this,
taskManager
)
);
if (mainConsumer instanceof ConsumerWrapper) {
((ConsumerWrapper) mainConsumer).subscribe(
topologyMetadata.allFullSourceTopicNames(),
new DefaultStreamsRebalanceListener(
log,
time,
streamsRebalanceData.get(),
this,
taskManager
)
);
} else {
((AsyncKafkaConsumer<byte[], byte[]>) mainConsumer).subscribe(
topologyMetadata.allFullSourceTopicNames(),
new DefaultStreamsRebalanceListener(
log,
time,
streamsRebalanceData.get(),
this,
taskManager
)
);
}
} else {
mainConsumer.subscribe(topologyMetadata.allFullSourceTopicNames(), rebalanceListener);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskCorruptedException;
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.Materialized;
import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
Expand Down Expand Up @@ -3926,38 +3925,6 @@ t2p1, new PartitionInfo(t2p1.topic(), t2p1.partition(), null, new Node[0], new N
);
}

@ParameterizedTest
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why have you removed these tests?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This PR removes public AsyncKafkaConsumer<byte[], byte[]> consumer() which is not needed any longer, but the test rely on it.

@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) {
final StreamsConfig config = new StreamsConfig(streamsConfigProps);
final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class);
Expand Down