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-12849: KIP-744 TaskMetadata ThreadMetadata StreamsMetadata as API #10840
KAFKA-12849: KIP-744 TaskMetadata ThreadMetadata StreamsMetadata as API #10840
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added some notes to help navigate the PR
* | ||
* @return {@link StreamsMetadata} for each {@code KafkaStreams} instances of this application | ||
*/ | ||
public Collection<StreamsMetadata> allRunningMetadata() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not so much convinced about this name, open to suggestions if you don't like it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about allStreamsMetadata
? I think that gets at the core difference here, which is that this returns metadata for all Streams instances.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What about metadataForAllStreamsClients()
? I think this makes it more explicit.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I decided to for for the pattern xxxForxxx to keep consistency among different changes.
It is now metadataForAllStreamsClients
but happy to change if anyone has reasons against it.
/** | ||
* Returns runtime information about the local threads of this {@link KafkaStreams} instance. | ||
* | ||
* @return the set of {@link ThreadMetadata}. | ||
*/ | ||
public Set<ThreadMetadata> localThreadsMetadata() { | ||
public Set<ThreadMetadata> threadsMetadata() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Similarly here, name is maybe not the best, happy to change it if there is a better alternative.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think localThreadMetadata
sounds better to me (actually even better than the original localThreadsMetadata
tbh), but I'm fine with threadsMetadata
too
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: I think localThreadMetadata
does not sound consistent since metadata of multiple stream threads is returned. What about metadataForLocalThreads()
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I decided to for for the pattern xxxForxxx to keep consistency among different changes.
It is now metadataForLocalThreads
but happy to change if anyone has reasons against it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@cadonna localThreadMetadata
still sounds more correct to me than localThreadsMetadata
. I really can't explain it other than to say that English is weird, and names/titles like this do not always follow the regular rules of grammar/plurals 🤷♀️
But actually I like your suggestion metadataForLocalThreads()
even better than any of them, SGTM
public class TaskMetadata { | ||
|
||
private final TaskId taskId; | ||
private final String taskId; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This reverts TaskMetadata to the state before KIP-740. As this TaskMetadata is deprecated, no need to do any type change here.
@@ -68,17 +61,8 @@ public TaskMetadata(final TaskId taskId, | |||
/** | |||
* @return the basic task metadata such as subtopology and partition id | |||
*/ | |||
public TaskId getTaskId() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same here, this reverts to the state prior KIP-740. Users are anyway discouraged to use this class.
/** | ||
* @return the basic task metadata such as subtopology and partition id | ||
*/ | ||
TaskId taskId(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Cleaned up API here, with proper naming convention for the new TaskMetada
As @cadonna, @ableegoldman and @guozhangwang showed some interest on the KIP, I guess maybe any of you can review this PR? |
Failure was:
On top of the common |
Following Konstantine's recommendation on the mailing list in regards of the dates for the release of Kafka 3.0.0, I'm pinging again potential reviewers for this KIP's implementation. Thanks a lot! |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Did a quick pass, looks good for the most part but I think we can take the opportunity to clean up the method names a bit. Lmk what you think of the suggestions
/** | ||
* Returns runtime information about the local threads of this {@link KafkaStreams} instance. | ||
* | ||
* @return the set of {@link ThreadMetadata}. | ||
*/ | ||
public Set<ThreadMetadata> localThreadsMetadata() { | ||
public Set<ThreadMetadata> threadsMetadata() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think localThreadMetadata
sounds better to me (actually even better than the original localThreadsMetadata
tbh), but I'm fine with threadsMetadata
too
* | ||
* @return {@link StreamsMetadata} for each {@code KafkaStreams} instances of this application | ||
*/ | ||
public Collection<StreamsMetadata> allRunningMetadata() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about allStreamsMetadata
? I think that gets at the core difference here, which is that this returns metadata for all Streams instances.
* @return {@link StreamsMetadata} for each {@code KafkaStreams} instances with the provide {@code storeName} of | ||
* this application | ||
*/ | ||
public Collection<StreamsMetadata> allMetadataForGivenStore(final String storeName) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't feel too strongly about any of these names, but it's probably best to try and keep them (a) consistent with each other, where possible, and (b) consistent with the return type. Assuming we go with something like allStreamsMetadata
for the new method above, maybe something like streamsMetadataForStore
would be appropriate for this one?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am +1 for streamsMetadataForStore
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I decided to for for the pattern xxxForxxx to keep consistency among different changes.
It is now streamsMetadataForStore
but happy to change if anyone has reasons against it.
@@ -27,10 +27,12 @@ | |||
|
|||
/** | |||
* Represents the state of a single task running within a {@link KafkaStreams} application. | |||
* @deprecated since 3.0, not intended for public use, use {@link org.apache.kafka.streams.TaskMetadata} instead. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
* @deprecated since 3.0, not intended for public use, use {@link org.apache.kafka.streams.TaskMetadata} instead. | |
* @deprecated since 3.0, use {@link org.apache.kafka.streams.TaskMetadata} instead. |
I think people might be a bit confused if we say this, just point them to the new interface
Collections.emptySet(), | ||
Collections.emptySet(), | ||
Collections.emptySet(), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: fix alignment (here and below change)
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
final org.apache.kafka.streams.processor.internals.TaskMetadataImpl that = (org.apache.kafka.streams.processor.internals.TaskMetadataImpl) o; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does this need to be fully qualified? There's only one TaskMetadataImpl
class right?
Collections.emptySet(), | ||
Collections.emptySet(), | ||
Collections.emptySet(), | ||
Collections.emptySet()); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: fix alignment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
/** | ||
* This function will return a set of the current TopicPartitions | ||
*/ | ||
Set<TopicPartition> topicPartitions(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you please use javadoc mark-up like @return
and @param
for the docs? Here and for the other methods.
*/ | ||
@Deprecated |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: If we deprecate the class, we do not need to deprecate the constructor anymore, right?
/** | ||
* This function will return the set of the {@link TaskMetadata} for the current active tasks | ||
*/ | ||
Set<TaskMetadata> activeTasks(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you use javadoc mark-up for the docs? Here and for the other methods.
/** | ||
* Returns runtime information about the local threads of this {@link KafkaStreams} instance. | ||
* | ||
* @return the set of {@link ThreadMetadata}. | ||
*/ | ||
public Set<ThreadMetadata> localThreadsMetadata() { | ||
public Set<ThreadMetadata> threadsMetadata() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: I think localThreadMetadata
does not sound consistent since metadata of multiple stream threads is returned. What about metadataForLocalThreads()
?
@@ -1972,12 +1971,6 @@ public void shouldAlwaysUpdateTasksMetadataAfterChangingState() { | |||
assertEquals(StreamThread.State.RUNNING.name(), metadata.threadState()); | |||
} | |||
|
|||
private void assertThreadMetadataHasEmptyTasksWithState(final ThreadMetadata metadata, final StreamThread.State state) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you for the clean up!
this.topicPartitions = topicPartitions; | ||
this.committedOffsets = committedOffsets; | ||
this.endOffsets = endOffsets; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since this class is read only, it makes sense to use Collections.unmodifiable*()
methods here to avoid modification of the contents of the member fields.
Unit tests as in StreamsMetadataTest
would also be good.
final Set<TaskMetadata> standbyTasks) { | ||
this.mainConsumerClientId = mainConsumerClientId; | ||
this.restoreConsumerClientId = restoreConsumerClientId; | ||
this.producerClientIds = producerClientIds; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think, you should use Collections.unmodifiableSet()
also here.
Also unit tests to verify the immutability as in StreamsMetadataTest
would be great!
*/ | ||
@Override | ||
public Set<String> stateStoreNames() { | ||
return Collections.unmodifiableSet(stateStoreNames); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it would be better to make the collections immutable in the constructor since they should also not be modified within this class (for now).
docs/streams/upgrade-guide.html
Outdated
<code>org.apache.kafka.streams.TaskMetadata</code> to be used instead. This change was introduced to better reflect the fact that <code>TaskMetadata</code> was not meant to be instantiated outside | ||
of Kafka codebase. | ||
Please note that the new <code>TaskMetadata</code> offers APIs that better represent the task id as an actual <code>TaskId</code> object instead of a String. Please migrate to the new | ||
<code>org.apache.kafka.streams.TaskMetadata</code> which offers these better methods, for example, by using the new <code>ThreadMetadata#getActiveTasks</code> and <code>ThreadMetadata#getStandbyTasks</code>. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
<code>org.apache.kafka.streams.TaskMetadata</code> which offers these better methods, for example, by using the new <code>ThreadMetadata#getActiveTasks</code> and <code>ThreadMetadata#getStandbyTasks</code>. | |
<code>org.apache.kafka.streams.TaskMetadata</code> which offers these better methods, for example, by using the new <code>ThreadMetadata#activeTasks</code> and <code>ThreadMetadata#standbyTasks</code>. |
d266b98
to
9c3ffe3
Compare
Implementation of KIP-744. Creates new Interfaces for TaskMetadata, ThreadMetadata, and StreamsMetadata, providing internal implementations for each of them. Deprecates current TaskMetadata, ThreadMetadata under o.a.k.s.processor, and SreamsMetadata under a.o.k.s.state. Updates references on internal classes from deprecated classes to new interfaces. Deprecates methods on KStreams returning deprecated ThreadMeatada and StreamsMetadta, and provides new ones returning the new interfaces. Update Javadocs referencing to deprecated classes and methods to point to the right ones.
9c3ffe3
to
a6a0bec
Compare
if (producerClientIds != null) { | ||
this.producerClientIds = Collections.unmodifiableSet(producerClientIds); | ||
} else { | ||
this.producerClientIds = Collections.emptySet(); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@cadonna, I needed to add this guard as ThreadMetadataTest was failing with NPE.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I could not reproduce this issue. the set of producer IDs should never be null
. If this is the case the test is wrong.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK, now I see. You meant StreamThreadTest
. There you need to setup the mock for the TaskManager
to return at least an empty set with expect(taskManager.producerClientIds()).andStubReturn(Collections.emptySet());
.
Needed to rebase as there were some conflicts with trunk, hence the force push. I applied the changes in separate commits:
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added some notes about the hash code and equals testing approach
} | ||
|
||
@Test | ||
public void shouldFollowEqualsAndHasCodeContract() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@cadonna For hash code and equals contract validation, I decided to keep it under one single test, and validate the positive cases and any of the reasons why it might fall into the negative case.
Please note that objects differing only on committed offsets, end offsets, and/or time curring idling started will be considered equals.
} | ||
|
||
@Test | ||
public void shouldFollowHashCodeAndEqualsContract() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@cadonna For hash code and equals contract validation, I decided to keep it under one single test, and validate the positive case and any of the reasons why it might fall into the negative case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I prefer to have separate tests, because when something fails you immediately know the cause. But if you like to keep like that, I am fine with it.
@@ -55,6 +62,63 @@ public void shouldNotAllowModificationOfInternalStateViaGetters() { | |||
assertTrue(isUnmodifiable(streamsMetadata.standbyStateStoreNames())); | |||
} | |||
|
|||
@Test | |||
public void shouldFollowHashCodeAndEqualsContract() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@cadonna For hash code and equals contract validation, I decided to keep it under one single test, and validate the positive case and any of the reasons why it might fall into the negative case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
see my other comment about separate tests.
@ableegoldman and @cadonna thanks a lot for your reviews! I think I addressed all of your comments. Let me know what do you think about them. |
I see some usual suspects on test failures, but
Is there some relation I'm missing, or should I create a new issue for this flaky test? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@jlprat Thank you for the update!
Here my comments!
Regarding the docs of the interfaces, could you provide at least one initial sentence for each javadoc? I know that the old classes were not documented that well. If you do not have time, we can also postpone the improvements of the docs to a separate PR and get this PR merged as soon as possible.
docs/streams/upgrade-guide.html
Outdated
<li>Users of <code>KafkaStreams#allMetadata</code> are meant to migrate to the new <code>KafkaStreams#allRunningMetadata</code>.</li> | ||
<li>Users of <code>KafkaStreams#allMetadataForStore(String)</code> are meant to migrate to the new <code>KafkaStreams#allMetadataForGivenStore(String)</code>.</li> | ||
<li>Users of <code>KafkaStreams#localThreadsMetadata</code> are meant to migrate to the new <code>KafkaStreams#threadsMetadata</code>.</li> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You need to adapt this text to the new method names.
localMetadata.set(new StreamsMetadataImpl(thisHost, | ||
Collections.emptySet(), | ||
Collections.emptySet(), | ||
Collections.emptySet(), | ||
Collections.emptySet() | ||
)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: it should be 4 not 8 spaces
localMetadata.set(new StreamsMetadataImpl(thisHost, | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet() | |
)); | |
localMetadata.set(new StreamsMetadataImpl( | |
thisHost, | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet() | |
)); |
final StreamsMetadata metadata = new StreamsMetadataImpl(hostInfo, | ||
activeStoresOnHost, | ||
activePartitionsOnHost, | ||
standbyStoresOnHost, | ||
standbyPartitionsOnHost); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
final StreamsMetadata metadata = new StreamsMetadataImpl(hostInfo, | |
activeStoresOnHost, | |
activePartitionsOnHost, | |
standbyStoresOnHost, | |
standbyPartitionsOnHost); | |
final StreamsMetadata metadata = new StreamsMetadataImpl( | |
hostInfo, | |
activeStoresOnHost, | |
activePartitionsOnHost, | |
standbyStoresOnHost, | |
standbyPartitionsOnHost | |
); |
if (producerClientIds != null) { | ||
this.producerClientIds = Collections.unmodifiableSet(producerClientIds); | ||
} else { | ||
this.producerClientIds = Collections.emptySet(); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I could not reproduce this issue. the set of producer IDs should never be null
. If this is the case the test is wrong.
public final static StreamsMetadataImpl NOT_AVAILABLE = new StreamsMetadataImpl(HostInfo.unavailable(), | ||
Collections.emptySet(), | ||
Collections.emptySet(), | ||
Collections.emptySet(), | ||
Collections.emptySet()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
public final static StreamsMetadataImpl NOT_AVAILABLE = new StreamsMetadataImpl(HostInfo.unavailable(), | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet()); | |
public final static StreamsMetadataImpl NOT_AVAILABLE = new StreamsMetadataImpl( | |
HostInfo.unavailable(), | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet(), | |
Collections.emptySet() | |
); |
public static final TaskMetadata TM_0 = new TaskMetadataImpl( | ||
TASK_ID_0, | ||
mkSet(TP_0_0, TP_1_0), | ||
mkMap(mkEntry(TP_0_0, 1L), mkEntry(TP_1_0, 2L)), | ||
mkMap(mkEntry(TP_0_0, 1L), mkEntry(TP_1_0, 2L)), | ||
Optional.of(3L)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit:
public static final TaskMetadata TM_0 = new TaskMetadataImpl( | |
TASK_ID_0, | |
mkSet(TP_0_0, TP_1_0), | |
mkMap(mkEntry(TP_0_0, 1L), mkEntry(TP_1_0, 2L)), | |
mkMap(mkEntry(TP_0_0, 1L), mkEntry(TP_1_0, 2L)), | |
Optional.of(3L)); | |
public static final TaskMetadata TM_0 = new TaskMetadataImpl( | |
TASK_ID_0, | |
mkSet(TP_0_0, TP_1_0), | |
mkMap(mkEntry(TP_0_0, 1L), mkEntry(TP_1_0, 2L)), | |
mkMap(mkEntry(TP_0_0, 1L), mkEntry(TP_1_0, 2L)), | |
Optional.of(3L) | |
); |
here and below.
|
||
/** | ||
* Compares the specified object with this ThreadMetadata. Returns {@code true} if and only if the specified object is | ||
* also a TaskMetadata and both {@code threadName()} are equal, {@code threadState()} are equal, {@code activeTasks()} contain the same |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TaskMetadata
-> ThreadMetadata
} | ||
|
||
@Test | ||
public void shouldFollowHashCodeAndEqualsContract() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I prefer to have separate tests, because when something fails you immediately know the cause. But if you like to keep like that, I am fine with it.
streamsMetadata = new StreamsMetadataImpl( | ||
HOST_INFO, | ||
STATE_STORE_NAMES, | ||
TOPIC_PARTITIONS, | ||
STAND_BY_STORE_NAMES, | ||
STANDBY_TOPIC_PARTITIONS |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: The indentation was actually correct before.
@@ -55,6 +62,63 @@ public void shouldNotAllowModificationOfInternalStateViaGetters() { | |||
assertTrue(isUnmodifiable(streamsMetadata.standbyStateStoreNames())); | |||
} | |||
|
|||
@Test | |||
public void shouldFollowHashCodeAndEqualsContract() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
see my other comment about separate tests.
Hi @cadonna I'll try to fix address the comments either today or tomorrow. Regarding the |
Yes, I agree! |
It includes - Split equality tests by cases - Add JavaDoc on methods - Fix formatting - Use proper assertThat instead of other means - Fix names, use the proper names consistently
Hi @cadonna I applied all the feedback you provided. I gave my best at adding some Extra sentences on the JavaDocs for the interfaces. Let me know if they suffice. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@jlprat I did a pass over the javadocs.
streams/src/main/java/org/apache/kafka/streams/StreamsMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/StreamsMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/StreamsMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/StreamsMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/StreamsMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/ThreadMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/ThreadMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/ThreadMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/ThreadMetadata.java
Outdated
Show resolved
Hide resolved
streams/src/main/java/org/apache/kafka/streams/ThreadMetadata.java
Outdated
Show resolved
Hide resolved
Co-authored-by: Bruno Cadonna <cadonna@apache.org>
@cadonna accepted your suggestions. Thanks a bunch!a |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@jlprat, Thanks a lot!
LGTM!
I am just waiting for the builds to finish and then I will merge. |
It seems that the test failures were RaftClusterTest and SslTransportLayerTest ones. Both groups being known flaky ones AFAIU. |
The test failures are unrelated and known to be flaky:
|
Thanks a lot @cadonna and @ableegoldman for the reviews! |
…PI (apache#10840) Implementation of KIP-744. Creates new Interfaces for TaskMetadata, ThreadMetadata, and StreamsMetadata, providing internal implementations for each of them. Deprecates current TaskMetadata, ThreadMetadata under o.a.k.s.processor, and SreamsMetadata under a.o.k.s.state. Updates references on internal classes from deprecated classes to new interfaces. Deprecates methods on KafkaStreams returning deprecated ThreadMeatada and StreamsMetadata, and provides new ones returning the new interfaces. Update Javadocs referencing to deprecated classes and methods to point to the right ones. Co-authored-by: Bruno Cadonna <cadonna@apache.org> Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Bruno Cadonna <cadonna@apache.org>
Implementation of KIP-744.
Creates new Interfaces for TaskMetadata, ThreadMetadata, and
StreamsMetadata, providing internal implementations for each of them.
Deprecates current TaskMetadata, ThreadMetadata under o.a.k.s.processor,
and SreamsMetadata under a.o.k.s.state.
Updates references on internal classes from deprecated classes to new interfaces.
Deprecates methods on KStreams returning deprecated ThreadMeatada and
StreamsMetadta, and provides new ones returning the new interfaces.
Update Javadocs referencing to deprecated classes and methods to point
to the right ones.
Committer Checklist (excluded from commit message)