-
Notifications
You must be signed in to change notification settings - Fork 13.9k
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-6849: add transformValues methods to KTable. #4959
KAFKA-6849: add transformValues methods to KTable. #4959
Conversation
Add overloads with Materialized
# Conflicts: # streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java # streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java # streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java # streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java # streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
@dguy , @guozhangwang , @mjsax can you guys take a look when you get a chance. The stricter the review the better. :D |
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 the PR @big-andy-coates . Made a pass over it and left some comments.
@@ -2906,6 +2906,7 @@ <h2><a class="toc-backref" href="#id7">Overview</a><a class="headerlink" href="# | |||
<tr class="row-even"><td><p class="first"><strong>Transform (values only)</strong></p> | |||
<ul class="last simple"> | |||
<li>KStream -> KStream</li> | |||
<li>KTable -> KTable</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.
Could we piggy back the docs change for KIP-149
here as well? E.g. in line 2912 below, add ValueTransformerWithKey
as well and emphasizing that the passed in key is for ready-only, but not mutable.
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.
@guozhangwang Not sure I follow exactly what doc changes you're after...
I can switch this paragraph to use ValueTransformerWithKey
rather than ValueTransformer
and call out that the key is readOnly. Is that what you're after?
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.
yup.
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, great.
@@ -382,6 +383,143 @@ | |||
*/ | |||
<KR> KStream<KR, V> toStream(final KeyValueMapper<? super K, ? super V, ? extends KR> mapper); | |||
|
|||
/** | |||
* Transform the value of each input record into a new value (with possible new type) of the output record. |
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 for first sentence: Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value.
Also we can add one more sentence comparing with mapValues
that this API allows users to have more flexible custom transformation logic that can rely on additional state stores and record context, plus they can add period functions via scheduling etc.
Ditto below.
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.
docs already calls out the potential use of punctuate
. I'll add something for more flexibility / state stores and record context.
|
||
/** | ||
* Transform the value of each input record into a new value (with possible new type) of the output record. | ||
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applies to each input |
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: add one more sentence that The resulted {@code KTable} is materialized into another state store (additional to the provided state store names) as specified by the user via {@link Materialized}, and is queryable through its given name.
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.
done
|
||
@Override | ||
public <K, V> void forward(final K key, final V value) { | ||
throw new StreamsException("ProcessorContext#forward() not supported."); |
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: not supported
is a tad confusing, maybe sth. like .. is not allowed, please specify the output record in
transform() return values
.
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.
@guozhangwang That makes sense in the current use of this class, i.e. it's only used from KStream and KTable transformValues
methods - but that might not always be the case.
Happy to change if you still think it better.
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.
@guozhangwang, what's your preference here?
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 it is really nit
I'm fine if you prefer it as is.
import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; | ||
import org.apache.kafka.streams.processor.ProcessorContext; | ||
|
||
/** |
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: remove this.
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.
Oops!
private <VR> KTable<K, VR> doTransformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> transformerSupplier, | ||
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized, | ||
final String... stateStoreNames) { | ||
Objects.requireNonNull(stateStoreNames, "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.
Hmm... stateStoreNames can indeed be null right?
name); | ||
} | ||
|
||
return new KTableImpl<>(builder, name, processorSupplier, sourceNodes, queryableStoreName, isQueryable); |
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.
If the resulted KTable is not materialized, we should 1) set the queryableStoreName
to the current queryableStoreName; and 2) set is Queryable to materialized != null && materialized.isQueryable()
. Please see doFilter
and doMapValues
for references.
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 spot! I did reference these for methods, but they've recently been changed. I'll update accordingly.
public void init(final ProcessorContext context) { | ||
parentGetter.init(context); | ||
|
||
valueTransformer.init(new ForwardingDisabledProcessorContext(context)); |
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 makes me thinking: we should let least document that users should not rely on the number of generated valueTransformers in their logic, because here, we will create two instances for getter and processor, hence we'd actually get 2N instances of transformers given N tasks, etc.
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.
@guozhangwang, is it 2N always, or either 1N or 2N depending on whether the downstream processor node is materialized or not?
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.
It is not 2N always, I think the point here is that the valueTransformer.transform() call could be triggered more than once for each record, because of 1) we may use a getter
that initialize a new value transformer, 2) we may send old values which calls transform
again on the old values. This is unfortunately not very intuitive to users, and we'd better warn them about that for now. And in the future we can consider how to address this (I've filed a JIRA for it)
private final String queryableName; | ||
private boolean sendOldValues = false; | ||
|
||
KTableTransformValues(final KTableImpl<K, ?, V> parent, |
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.
For KTableTransformValuesProcessor
that extends AbstractProcessor
, we should override its close()
call as:
@Override
public void close() {
valueTransformer.close();
}
Similarly we should close the valueTransformer
in Getter
as well. Unfortunately we do not have a close()
in KTableValueGetter
yet. If there is no better way to call valueTransformer.close()
when the topology is closing we should then add this function and make sure it is called when the dependent processor node is being closed.
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.
Can't see a better way, so adding close()
to KTableValueGetter and wiring that up.
@@ -162,6 +164,43 @@ class KTable[K, V](val inner: KTableJ[K, V]) { | |||
def toStream[KR](mapper: (K, V) => KR): KStream[KR, V] = | |||
inner.toStream[KR](mapper.asKeyValueMapper) | |||
|
|||
/** | |||
* Transform the value of each input record into a new value (with possible new type) of the output record. |
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.
Ditto for the scala doc as above.
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.
done
Also could you update the |
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 not look into the tests in detail yet.
@@ -220,6 +223,47 @@ public String queryableStoreName() { | |||
return doMapValues(mapper, new MaterializedInternal<>(materialized, builder, MAPVALUES_NAME)); | |||
} | |||
|
|||
@Override | |||
public <VR> KTable<K, VR> transformValues(ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> transformerSupplier, String... 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.
nit: add final
nit: one argument per line (line too long)
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.
done.
} | ||
|
||
private <VR> KTable<K, VR> doTransformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> transformerSupplier, | ||
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized, |
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 indention
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.
done.
private <VR> KTable<K, VR> doTransformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> transformerSupplier, | ||
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized, | ||
final String... stateStoreNames) { | ||
Objects.requireNonNull(stateStoreNames, "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.
Does this do the trick? The type is String[]
and can only be null
if somebody calls via
String[] stores = null;
table.transformValues(...., stores);
table.transformValues(...., null); // this would not trigger the check
Should better step through the array and check each entry for null
? Or is this intended and we rely on connectProcessorAndStateStores
to do this check?
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.
Ah I now remembered: to guard against (... , null)
we should follow the way in InternalTopologyBuilder#addStateStore
. i.e.:
if (stateStoreNames != null) {
for (final String stateStoreName : stateStoreNames) {
Objects.requireNonNull(processorName, "store name must not be null");
}
}
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.
Yes, this check is to ensure someone hasn't done something daft like:
String[] stores = null;
table.transformValues(...., stores);
If someone passes in a null store name it is caught later.
} | ||
|
||
@Override | ||
public void process(K key, Change<V> change) { |
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: add final
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.
done.
@Override | ||
public void process(K key, Change<V> change) { | ||
final V1 newValue = computeValue(key, change.newValue, valueTransformer); | ||
final V1 oldValue = sendOldValues ? computeValue(key, change.oldValue, valueTransformer) : null; |
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 about this... If the operator is stateful, the computation might depend on the state. Thus, this might not compute the correct oldValue
...
On the other hand, I am wondering if we need the old value in the first place -- also, the old value would be store in the result KTable state (if there is any) -- thus, maybe using the result KTable state might be the better option (maybe we even need to force a materialization if we need to send oldValue
.
WDYT?
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.
Hmm good question ... I think we still need the old values, but I also agrees that we can just get it from the getter
than re-computing it. Note that if the result KTable is not materialized, then getter
will rely on parent's materialized store as well which is still an issue.
This is a known issue that I have been thinking to fix, but the scope could be quite large. Maybe we can create a JIRA for fixing this separately.
I will create a JIRA for now.
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.
There is plenty of scope for computing the wrong old value here. I've changed the code to make the computation of old value very explicit.
Materialized tables are easy. Where unmaterialized tables are mixed with stateful transformations it is easy for users to break downstream aggregates. (There's a test that shows this). I've documented this in the Java doc. I think it still makes sense to have the ability to pass in additional stateStoreNames for the non-materialized transformValues
method call. Not allowing them would remove a big 'gotcha' but may be overly restrictive too.
No doubt the wording of the Java docs around this issue will need finessing, and you may prefer me to remove the 'prove this breaks tests', but just let me know.
@@ -320,9 +320,9 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { | |||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) | |||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues` | |||
*/ | |||
def transformValues[VR](valueTransformerWithKeySupplier: ValueTransformerWithKeySupplier[K, V, VR], | |||
def transformValues[VR](valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR], |
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.
Why this renaming?
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.
Intellij madness that I missed. Fixing...
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.
Ah, no, not madness - I changed it to match the scala doc parameter name.
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.
Maybe we should update the doc parameter name 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.
Sure, I just prefer short param names.
Created https://issues.apache.org/jira/browse/KAFKA-6903 as per @mjsax 's comment |
@big-andy-coates could you rebase? There is a recent commit changing
|
Hey @guozhangwang @mjsax, I've pushed some commits. The size of the PR has grown somewhat due to adding the I think all issues have been addressed. Likely contentious points are:
|
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.
Left some more comments.
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) | ||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues` | ||
*/ | ||
def transformValues[VR](valueTransformerWithKeySupplier: ValueTransformerWithKeySupplier[K, V, VR], |
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.
There are some proposals on refactoring the scala api to make better leverage on its type inference: #5019
I think to be consistent with that PR the API should be
transformValues[VR](valueTransformerWithKeySupplier: ValueTransformerWithKeySupplier[K, V, VR]) (materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]],
stateStoreNames: String*): KTable[K, VR])
I will let @joan38 to chime in here as well:
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.
It's been a while since i've looked at Scala, can I suggest such changes are left to the linked PR? i.e. we try and merge this one first, as there seems to be open questions of the other PR.
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.
It should be ok here since valueTransformerWithKeySupplier
doesn't have parameters that depends on another.
public void init(final ProcessorContext context) { | ||
super.init(context); | ||
|
||
oldValueGetter.init(context); |
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 seems an overkill to me: for old values, we only need change.oldValue
, and actually all of its implementations either do not need the value, or only look for change.oldValue
.
We can probably simply it in line 132, just as:
oldValue = queryableName ? /* get old value from materialized store*/ : computeValue(key, change.oldValue, valueTransformer);
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.
Actually I think using the materialized store to get the old value
can be a separate optimization PR by itself: we should consider doing this for filter, mapValues, transformValues, etc, all operators.
To keep the scope / loc of this PR small, we can consider only use computeValue(key, change.oldValue, valueTransformer)
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 does mean backing out changes that will make this PR less correct. I'm happy to do it, but my opinion would be to leave it in.
With regards to the overkill... yep, you're absolutely right... doh - not sure how I missed that- sorting.
@big-andy-coates Updated https://issues.apache.org/jira/browse/KAFKA-6903 for the optimization on sending old values. |
@guozhangwang @mjsax changes pushed.
|
# Conflicts: # streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinValueGetter.java
@mjsax - are you happy? Is so, can you merge please? |
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 pushed during my review... submitting what I have a continue (cannot add not comments to this review)
/** | ||
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value, | ||
* (with possibly new type). | ||
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applies to each input |
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.
is applie[d]
(please double check if this is c&p error and must be fixed somewhere else, 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.
done everywhere.
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!
* This is similar to {@link #mapValues(ValueMapperWithKey)}, but more flexible, allowing access to additional state-stores, | ||
* and access to the {@link ProcessorContext}. | ||
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can be observed and additional | ||
* periodic actions get be performed. |
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.
get
-> can
?
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.
c&P again - done everywhere.
* new ValueTransformerWithKeySupplier() { | ||
* ValueTransformerWithKey get() { | ||
* return new ValueTransformerWithKey() { | ||
* private StateStore 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.
Should we update this to be a KeyValueStore
to align with the registered store from the example above?
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.
Sure. (Shame a cast is involved here :( )
* private StateStore state; | ||
* | ||
* void init(ProcessorContext context) { | ||
* this.state = context.getStateStore("myValueTransformState"); |
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.
might need to insert a cast (cf. comment above)
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.
Yep, done.
/** | ||
* Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value, | ||
* (with possibly new type). | ||
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applies to each input |
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.
applie[d]
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.
done
} | ||
|
||
@Override | ||
public Integer transform(String readOnlyKey, String value) { |
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: add final
} | ||
} | ||
|
||
private static class StatefulTransformer implements ValueTransformerWithKey<String, String, Integer> { |
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.
maybe rename s.th. it describes what it does?
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 it does isn't as important as the fact its stateful vs stateless IMHO.
} | ||
} | ||
|
||
private static class StatelessTransformer implements ValueTransformerWithKey<String, String, Integer> { |
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.
maybe rename s.th. it describes what it does?
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.
as above.
driver = new TopologyTestDriver(builder.build(), props()); | ||
|
||
driver.pipeInput(recordFactory.create(INPUT_TOPIC, "A", "a", 0L)); | ||
driver.pipeInput(recordFactory.create(INPUT_TOPIC, "A", "aa", 0L)); |
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.
maybe use "a", "b", "c" as values, as the transformer counts the number of calls to process
(for better distinction with next 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.
yer, or maybe just 'ignored'
public void shouldCalculateCorrectOldValuesIfNotStatefulEvenIfNotMaterialized() { | ||
|
||
builder.addStateStore(storeBuilder(STORE_NAME)) | ||
.table(INPUT_TOPIC, CONSUMED) |
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: indention
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.
done.
@Test | ||
public void shouldCalculateCorrectOldValuesIfNotStatefulEvenIfNotMaterialized() { | ||
|
||
builder.addStateStore(storeBuilder(STORE_NAME)) |
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.
store not used
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.
done.
|
||
@Test | ||
public void shouldCalculateCorrectOldValuesIfNotStatefulEvenIfNotMaterialized() { | ||
|
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: remove empty line
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.
done.
|
||
@Test | ||
public void willUnfortunatelyCalculateIncorrectOldValuesIfStatefulAndNotMaterialized() { | ||
builder.addStateStore(storeBuilder(STORE_NAME)) |
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.
store not used
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.
done
@Test | ||
public void willUnfortunatelyCalculateIncorrectOldValuesIfStatefulAndNotMaterialized() { | ||
builder.addStateStore(storeBuilder(STORE_NAME)) | ||
.table(INPUT_TOPIC, CONSUMED) |
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 indention
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.
done.
driver.pipeInput(recordFactory.create(INPUT_TOPIC, "A", "aaa", 0L)); | ||
|
||
assertThat(output(), not(hasItems("A:1", "A:0", "A:2", "A:0", "A:3"))); | ||
// Output more likely to be "A:1", "A:-2", "A:0", "A:-5", "A:-1" |
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.
Is this test useful?
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.
well, indeed. I did call it out. It does highlight the issue and, if it starts to fail, there is a doc update needed. Happy to remove though. (I agree its a bit weird.)
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.
The idea with the doc change is good -- maybe put a comment about this -- otherwise, when it fails, we might not remember the purpose of this test.
final List<String> expectedStoredNames) { | ||
final List<String> missing = new ArrayList<>(); | ||
|
||
for (String storedName : expectedStoredNames) { |
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: add final
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.
done.
return new ValueTransformerWithKey<K, V, V>() { | ||
|
||
@Override | ||
public void init(ProcessorContext context) { |
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: add final
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.
done.
} | ||
|
||
@Override | ||
public V transform(K readOnlyKey, V value) { |
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: add final
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.
done.
import org.apache.kafka.streams.processor.ProcessorContext; | ||
|
||
public class NoOpInternalValueTransformer<K, V> implements ValueTransformerWithKeySupplier<K, V, V> { | ||
public ProcessorContext context; |
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 think that the context should be a member of the supplier, but rather of the transformer -- otherwise, multiple transformers share the same context ?
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 agree, but this is a simple test class and switching it to be a member of the transformer muddies the test cases. How strongly do you feel about this?
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 see... Maybe rename to SingletonNoOpInternalValueTransformer
and only create one ValueTransformerWithKey
instance? This makes it more clear why it's designed this way? From the context of the class itself it's unclear -- only makes sense if you check it's usage in the test. Would be good to be self-descriptive IMHO.
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.
Done - also, 'Internal' part dropped inline with change that dropped the InternalValueTranformerWithKey
interface
@mjsax nits fixed ;) |
retest this please |
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.
LGTM. Thanks for the PR.
See the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-292%3A+Add+transformValues%28%29+method+to+KTable This PR adds the transformValues method to the KTable interface. The semantics of the call are the same as the methods of the same name on the KStream interface. Fixes KAFKA-6849 Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
See the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-292%3A+Add+transformValues%28%29+method+to+KTable
This PR adds the
transformValues
method to theKTable
interface. The semantics of the call are the same as the methods of the same name on theKStream
interface.Fixes KAFKA-6849