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

KAFKA-16331: remove EOSv1 from StreamsProducer #17212

Merged
merged 4 commits into from
Sep 23, 2024

Conversation

mjsax
Copy link
Member

@mjsax mjsax commented Sep 16, 2024

No description provided.

@mjsax mjsax added the streams label Sep 16, 2024
@@ -203,11 +198,6 @@ public static Map<TopicPartition, ListOffsetsResultInfo> fetchEndOffsets(final C
return getEndOffsets(fetchEndOffsetsFuture(partitions, adminClient));
}

public static String extractThreadId(final String fullThreadName) {
Copy link
Member Author

Choose a reason for hiding this comment

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

side cleanup -- unused -- removing

eosAlphaMockClientSupplier.setCluster(cluster);
eosAlphaMockClientSupplier.setApplicationIdForProducer("appId");
eosAlphaStreamsProducer =
eosMockClientSupplier.setCluster(cluster);
Copy link
Member Author

Choose a reason for hiding this comment

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

Many EOS tests where not EOSv1 vs EOSv2 specific, but used eosAlphaXxx variables -- updating these to use eosBetaXxx (renamed to just eosXxxx) -- more of this below.

@@ -191,7 +191,7 @@ public static <K, V> KeyValueStoreTestDriver<K, V> create(final Serializer<K> ke
private final Set<K> flushedRemovals = new HashSet<>();
private final List<KeyValue<byte[], byte[]>> restorableEntries = new LinkedList<>();

private final InternalMockProcessorContext context;
private final InternalMockProcessorContext<?, ?> context;
Copy link
Member Author

Choose a reason for hiding this comment

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

side cleanup

@@ -266,7 +265,7 @@ public <K1, V1> void send(final String topic,
stateDir.mkdirs();
stateSerdes = serdes;

context = new InternalMockProcessorContext(stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector, null) {
context = new InternalMockProcessorContext<Object, Object>(stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector, null) {
Copy link
Member Author

Choose a reason for hiding this comment

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

side cleanup

@@ -447,7 +446,7 @@ private StreamTask createStreamsTask(final StreamsConfig streamsConfig,
topology
);
final StreamsMetricsImpl streamsMetrics = new MockStreamsMetrics(metrics);
final InternalProcessorContext context = new ProcessorContextImpl(
final InternalProcessorContext<?, ?> context = new ProcessorContextImpl(
Copy link
Member Author

Choose a reason for hiding this comment

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

side cleanup

@mjsax mjsax force-pushed the kafka-16331-eosv1-streamsproducer branch from 09744d7 to 32d8ae0 Compare September 20, 2024 23:01
@mjsax mjsax changed the title Kafka 16331 eosv1 streamsproducer KAFKA-16331: remove EOSv1 from StreamsProducer Sep 20, 2024
Copy link
Contributor

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

@mjsax thanks for this patch. Could you please fix the build error:

[ant:checkstyle] [ERROR] /home/chia7712/project/kafka/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java:46:8: Unused import - org.apache.kafka.streams.processor.TaskId.

@@ -318,14 +314,7 @@ Map<MetricName, Metric> producerMetrics() {
}

Set<String> producerClientIds() {
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we use String instead of Set<String>> for now?

Copy link
Member Author

@mjsax mjsax Sep 22, 2024

Choose a reason for hiding this comment

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

Yes, but not "completely". The ThreadMetadata interface is public API, and thus it's impl cannot be changed. I can include as much cleanup as possible thought.

Would need a KIP to change it -- not sure if worth doing at this point thought...

@@ -191,7 +191,7 @@ public static <K, V> KeyValueStoreTestDriver<K, V> create(final Serializer<K> ke
private final Set<K> flushedRemovals = new HashSet<>();
private final List<KeyValue<byte[], byte[]>> restorableEntries = new LinkedList<>();

private final InternalMockProcessorContext context;
private final InternalMockProcessorContext<?, ?> context;
private final StateSerdes<K, V> stateSerdes;

@SuppressWarnings("unchecked")
Copy link
Contributor

Choose a reason for hiding this comment

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

this can be removed

@chia7712 chia7712 merged commit 9685aa7 into apache:trunk Sep 23, 2024
9 of 10 checks passed
@mjsax mjsax deleted the kafka-16331-eosv1-streamsproducer branch September 23, 2024 22:55
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants