KAFKA-7875: Add KStream.flatTransformValues#6424
Conversation
|
I was not able to use the following signature for When I tried to use this signature, I ran into compilation issues caused by generics. I was able to fix the compilation issues, but then I run again into the issue with value transformer suppliers that would not compile as lambda functions but would compile as anonymous inner classes. See the following discussion that we had about this issue during the implementation of With this signature I did not run into the lambda function compilation issue. Here more tests would be needed to check if all works as expected. Replacing Kafka's transformer suppliers with Java's supplier would solve the issue and it would additionally reduce the code to maintain. The drawbacks are as follows:
Are there also other drawbacks that hinder to use the Java's supplier in the KStream API that I am not aware of? Finally, because of these issues I was also not able to specify |
|
Hey, @cadonna , Sorry for the slow reply, I've pulled down your branch and played around with it a bit. I've reached the same conclusion as you, the generic type inference behaves differently between our I'm not 100% on my type algebra, but I think that the two should be equivalent, so I think we're just coping with the limitations of Java's generic type inference (again). In that case, without a deep knowledge of the java compiler's implementation, we're reduced to figuring out our options experimentally. Here's what I found (acknowledging that it's pretty much the same as you said above): All the examples are using this stream: final KStream<Object, Object> asdf = builder.stream("asdf");For Our supplier <VR> KStream<K, VR> flatTransformValuesA(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends Iterable<? extends VR>> valueTransformerSupplier,
final String... stateStoreNames);Which we can use as: final KStream<Object, Number> streamA1 =
asdf.flatTransformValuesA(new ValueTransformerWithKeySupplier<Object, Object, List<Integer>>() {
@Override
public ValueTransformerWithKey<Object, Object, List<Integer>> get() {
return new ValueTransformerWithKey<Object, Object, List<Integer>>() {
@Override public void init(final ProcessorContext context) { }
@Override
public List<Integer> transform(final Object readOnlyKey, final Object value) {
return asList(1, 2, 3);
}
@Override public void close() { }
};
}
});
final KStream<Object, Number> streamA2 =
asdf.flatTransformValuesA((ValueTransformerWithKeySupplier<Object, Object, List<Integer>>) () -> new ValueTransformerWithKey<Object, Object, List<Integer>>() {
@Override public void init(final ProcessorContext context) { }
@Override
public List<Integer> transform(final Object readOnlyKey, final Object value) {
return asList(1, 2, 3);
}
@Override public void close() { }
});Java Supplier <VR> KStream<K, VR> flatTransformValuesB(final Supplier<ValueTransformerWithKey<? super K, ? super V, ? extends Iterable<? extends VR>>> valueTransformerSupplier,
final String... stateStoreNames);Which we can use as: final KStream<Object, Number> streamB1 =
asdf.flatTransformValuesB(new Supplier<ValueTransformerWithKey<? super Object, ? super Object, ? extends Iterable<? extends Number>>>() {
@Override
public ValueTransformerWithKey<? super Object, ? super Object, ? extends Iterable<? extends Number>> get() {
return new ValueTransformerWithKey<Object, Object, List<Integer>>() {
@Override public void init(final ProcessorContext context) {}
@Override
public List<Integer> transform(final Object readOnlyKey, final Object value) {
return asList(1, 2, 3);
}
@Override public void close() {}
};
}
});
final KStream<Object, Number> streamB2 =
asdf.flatTransformValuesB(() -> new ValueTransformerWithKey<Object, Object, List<Integer>>() {
@Override public void init(final ProcessorContext context) {}
@Override
public List<Integer> transform(final Object readOnlyKey, final Object value) {
return asList(1, 2, 3);
}
@Override public void close() {}
});ConclusionThey both have downsides.
I see the essential dilemma as which implementation style we want to facilitate: lambda vs. subclass. Worth noting that in all cases, we're talking about how to declare the supplier, not the transformer, which must always be a declared class. Also worth noting, I'm using |
|
Thank you @vvcephei for your experiments. I am im favour of using the following signature: This is also the one used in this PR. This is the more restrictive one and it is not consistent with the signatures of transformValues and flatMapValues. However, the signature accepts lambda functions and classes without strange declarations on user side which in my opinion is most important. Consolidation of the signatures -- especially of the generics bounds -- seems to me a bigger issue that should be addressed on its own, if needed at all. |
|
I think I buy the arguments for allowing lambdas naturally with the cost of inconsistency on signatures (the only inconsistency comes from |
| final String... stateStoreNames); | ||
|
|
||
| /** | ||
| * Transform the value of each input record into zero or more new values (with possible new |
There was a problem hiding this comment.
nit: with possible new type -> with possibly a new type
| * Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()} | ||
| * the processing progress can be observed and additional periodic actions can be performed. | ||
| * <p> | ||
| * In order to assign a state, the state must be created and registered beforehand: |
| * | ||
| * KStream outputStream = inputStream.flatTransformValues(new ValueTransformerSupplier() { ... }, "myValueTransformState"); | ||
| * }</pre> | ||
| * Within the {@link ValueTransformer}, the state is obtained via the {@link ProcessorContext}. |
| * Furthermore, via {@link org.apache.kafka.streams.processor. Punctuator#punctuate()} the processing progress can | ||
| * be observed and additional periodic actions can be performed. | ||
| * <p> | ||
| * In order to assign a state, the state must be created and registered beforehand: |
| * | ||
| * KStream outputStream = inputStream.flatTransformValues(new ValueTransformerWithKeySupplier() { ... }, "myValueTransformState"); | ||
| * }</pre> | ||
| * Within the {@link ValueTransformerWithKey}, the state is obtained via the {@link ProcessorContext}. |
|
|
||
| @Test | ||
| public void shouldTransform() throws Exception { | ||
| public void shouldTransformValuesWithValueTransformerWithKey() throws Exception { |
| } | ||
|
|
||
| @Test | ||
| public void shouldTransformValuesWithValueTransformerWithoutKey() throws Exception { |
| } | ||
|
|
||
| @Test | ||
| public void shouldFlatTransformValuesWithKey() throws Exception { |
| } | ||
|
|
||
| @Test | ||
| public void shouldFlatTransformValuesWithValueTransformerWithoutKey() throws Exception { |
| inputValue = 10; | ||
| valueTransformer = mock(ValueTransformerWithKey.class); | ||
| context = strictMock(ProcessorContext.class); | ||
| processor = new KStreamFlatTransformValuesProcessor<Integer, Integer, String>(valueTransformer); |
There was a problem hiding this comment.
nit: can use new KStreamFlatTransformValuesProcessor<>
|
FWIW I also agree with using the signature supporting lamdas directly as IMHO the inconsistencies we have observed don't outweigh keeping things more straight forward for users. |
- corrected javadocs in KStream - minor refactorings in tests
| * A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input | ||
| * record value and computes zero or more new values. | ||
| * Thus, an input record {@code <K,V>} can be transformed into output records {@code <K:V'>, <K:V''>, ...}. | ||
| * This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}). |
There was a problem hiding this comment.
| * This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}). | |
| * This is a stateful record-by-record operation (see {@link #mapValues(ValueMapper) mapValues()}). |
There was a problem hiding this comment.
Throughout the javadocs in KStreamImpl, cf. is used. Do you want me to change all of them in a new PR? Are there other opinions on this?
There was a problem hiding this comment.
I don't know if we have a documentation style guide, but it's generally more accessible if we don't use Latin in our docs. No need to go making sweeping changes, but there's also no need to perpetuate an anti-pattern.
There was a problem hiding this comment.
I don't know if we have a documentation style guide
I don't think we have one.
but it's generally more accessible if we don't use Latin in our docs.
This is surprising to me. I thought cf. is a standard abbreviation that everybody knows...
I would still use cf. in this PR for consistency reasons. If you prefer see, just create a ticket or open a new PR. We can also update this on a per-class basis, but I would prefer to keep it consistent within a class.
| transformNode.setValueChangingOperation(true); | ||
| builder.addGraphNode(this.streamsGraphNode, transformNode); | ||
|
|
||
| // cannot inherit value serde |
There was a problem hiding this comment.
should there be a way to set it, then?
There was a problem hiding this comment.
Setting the serde to null seems to be a common pattern in KStreamImpl if the operation on the stream cannot guarantee that the current serde still fits for the newly generated stream. Have a look at the other non-terminal operations.
There was a problem hiding this comment.
Sorry, my comment was too terse. What I meant to ask was, if this is a value-changing operation, should there be an overload that allows users to add a value serde, so that downstream operations can have the serde available?
Off the top of my head, I don't think there are any other value-changing operations that don't have an option to set a serde. (but I could be wrong...)
There was a problem hiding this comment.
Aaand, nevermind. I was thinking of KTable. Carry on.
|
Java 8 passed, Java 11 failed with known flaky test retest this please |
|
Same as above. Retest this please. |
| * A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input | ||
| * record value and computes zero or more new values. | ||
| * Thus, an input record {@code <K,V>} can be transformed into output records {@code <K:V'>, <K:V''>, ...}. | ||
| * This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}). |
There was a problem hiding this comment.
I don't know if we have a documentation style guide
I don't think we have one.
but it's generally more accessible if we don't use Latin in our docs.
This is surprising to me. I thought cf. is a standard abbreviation that everybody knows...
I would still use cf. in this PR for consistency reasons. If you prefer see, just create a ticket or open a new PR. We can also update this on a per-class basis, but I would prefer to keep it consistent within a class.
| * context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..)); | ||
| * } | ||
| * | ||
| * NewValueType transform(V value) { |
There was a problem hiding this comment.
NewValueType -> List<NewValueType> ? (Similar in return below.)
| * Within the {@link ValueTransformer}, the state store is obtained via the {@link ProcessorContext}. | ||
| * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, | ||
| * a schedule must be registered. | ||
| * The {@link ValueTransformer} must return a list of values in {@link ValueTransformer#transform(Object) |
There was a problem hiding this comment.
must return a list of values -- could this be miss leading, that people think the need to return a List? (Similar next sentence below.)
| * Within the {@link ValueTransformerWithKey}, the state store is obtained via the {@link ProcessorContext}. | ||
| * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, | ||
| * a schedule must be registered. | ||
| * The {@link ValueTransformer} must return a list of values in {@link ValueTransformer#transform(Object) |
| * a schedule must be registered. | ||
| * The {@link ValueTransformer} must return a list of values in {@link ValueTransformer#transform(Object) | ||
| * transform()} or {@code null}. | ||
| * If the return value is an empty list or {@null} no records are emitted. |
| * context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..)); | ||
| * } | ||
| * | ||
| * NewValueType transform(K readOnlyKey, V value) { |
| * | ||
| * NewValueType transform(K readOnlyKey, V value) { | ||
| * // can access this.state and use read-only key | ||
| * return new NewValueType(readOnlyKey); // or null |
| builder.addGraphNode(this.streamsGraphNode, transformNode); | ||
|
|
||
| // cannot inherit value serde | ||
| return new KStreamImpl<>(name, keySerde, null, sourceNodes, this.repartitionRequired, transformNode, builder); |
There was a problem hiding this comment.
nit: don't use this is not required.
| } | ||
|
|
||
| @Test | ||
| public void shouldInitialiseFlatTransformValuesProcessor() { |
- corrected mistake in example in javadocs - improved javadocs for flatTransformValues and transformValues - corrected typo in test method name
1 similar comment
|
Java 8 failed with a known flaky test retest this please |
|
Merged #6424 into trunk |
Adds flatTrasformValues methods in KStream Adds processor supplier and processor for flatTransformValues Improves API documentation of transformValues Reviewers: Matthias J. Sax <mjsax@apache.org>, John Roesler <john@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
This contribution is my original work and I license the work to the project under the project's open source license.
More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.
Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.
Committer Checklist (excluded from commit message)