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-8410: Part 1: processor context bounds #8414
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -69,7 +69,7 @@ public interface ValueTransformer<V, VR> { | |
* @throws IllegalStateException If store gets registered after initialization is already finished | ||
* @throws StreamsException if the store's change log does not contain the partition | ||
*/ | ||
void init(final ProcessorContext context); | ||
void init(final ProcessorContext<Void, Void> context); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. As proposed, Void types indicate that you can't forward anything. |
||
|
||
/** | ||
* Transform the given value to a new value. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,6 +21,7 @@ | |
import org.apache.kafka.streams.kstream.ValueJoiner; | ||
import org.apache.kafka.streams.processor.AbstractProcessor; | ||
import org.apache.kafka.streams.processor.ProcessorContext; | ||
import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; | ||
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
@@ -49,11 +50,11 @@ class KStreamKTableJoinProcessor<K1, K2, V1, V2, R> extends AbstractProcessor<K1 | |
} | ||
|
||
@Override | ||
public void init(final ProcessorContext context) { | ||
public void init(final ProcessorContext<Object, Object> context) { | ||
super.init(context); | ||
metrics = (StreamsMetricsImpl) context.metrics(); | ||
droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); | ||
valueGetter.init(context); | ||
valueGetter.init(new ForwardingDisabledProcessorContext(context)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Even in this first PR, we expose some bugs. In this case, ValueGetters and ValueTransformers shouldn't be able to forward anything, but nothing previously prevented them from doing it. Now, the new bounds force us to wire in a context that has Void bounds. |
||
} | ||
|
||
@Override | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,11 +24,11 @@ | |
import org.apache.kafka.streams.state.internals.CacheFlushListener; | ||
|
||
class SessionCacheFlushListener<K, V> implements CacheFlushListener<Windowed<K>, V> { | ||
private final InternalProcessorContext context; | ||
private final ProcessorNode myNode; | ||
private final InternalProcessorContext<Object, Object> context; | ||
private final ProcessorNode<?, ?> myNode; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You'll notice that sometimes, I use wildcard ( Punchline, it doesn't really matter, but there are considerations that led me to use |
||
|
||
SessionCacheFlushListener(final ProcessorContext context) { | ||
this.context = (InternalProcessorContext) context; | ||
SessionCacheFlushListener(final ProcessorContext<Object, Object> context) { | ||
this.context = (InternalProcessorContext<Object, Object>) context; | ||
myNode = this.context.currentNode(); | ||
} | ||
|
||
|
@@ -37,7 +37,7 @@ public void apply(final Windowed<K> key, | |
final V newValue, | ||
final V oldValue, | ||
final long timestamp) { | ||
final ProcessorNode prev = context.currentNode(); | ||
final ProcessorNode<?, ?> prev = context.currentNode(); | ||
context.setCurrentNode(myNode); | ||
try { | ||
context.forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(key.window().end())); | ||
|
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.
In the KIP, I proposed to leave the existing Processor and Transformer init methods with a raw ProcessorContext argument. This will give all users a rawtypes warning with no recourse. Instead, in this PR, I'm adding
<Object, Object>
bounds instead. This at least allows users to also add the generics if they desire, while not changing the practical bounds at all. If this proves acceptable, I'll update the KIP.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 sure if I understand this. Can you elaborate?
Btw: to we think we will have any path forward to let user specify the actually return type (that is currently only
R
but does not specify the actually key or value type), such that we would pass inProcessorContext<K, V>
? Can't remember if we discussed this on the KIP (just a side question).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.
Good question. Yes, it's been forever, but we did (myself, I had to look back over the KIP to figure this out when I wrote the comment).
The idea is to discuss separately how to migrate to a version of Transformer that allows bounding the output types. The ticket for this is https://issues.apache.org/jira/browse/KAFKA-8396. This KIP was already sweeping enough in its design and implementation scope that we wanted to give Transformer its own discussion. Accordingly, this KIP only has to deal with the Processor API change and the consequences that are necessary/convenient to tackle at the same time.
Part of the reason to give Transformer its own discussion is that there are other opportunities to simplify
transform()
use cases (see the ticket for more info), so it really would be its own conversation.What I was talking about above is that I proposed in this KIP only to leave Transformer completely alone, so this would still be a raw type:
void init(final ProcessorContext context)
. But what I realized just now is that anyone who implements the interface will get a "raw types" warning from javac, and there is nothing they can do about it but ignore it. If they try to add the equivalent<Object, Object>
or<?, ?>
parameters, their implementation would clash with our interface.OTOH, if we go ahead and add the parameters as I proposed above, then at least they can also add the
<Object, Object>
parameters to their code to resolve the warning. They can not bound the types tighter than that, and neither can we, until we pick up KAFKA-8396. So, it doesn't provide any additional type safety, just a way to resolve the warning without resolving to ignoring it. AFAICT, there is no way to compatibly migrate the interface to accommodate greater type safety. KAFKA-8396 will have to include deprecating something in any 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.
Thanks for clarification.