mirror of https://github.com/apache/kafka.git
KAFKA-18917: TransformValues throws NPE (#19089)
When `transformValues` is used with a `Materialized` instance, but without a queryable name, a `NullPointerException` is thrown. To preserve the semantics present in 3.9, we need to avoid materialization when a queryable name is not present. Reviewers: Bruno Cadonna <cadonna@apache.org>
This commit is contained in:
parent
8533c43c9d
commit
295760d3eb
|
@ -448,8 +448,12 @@ public class KTableImpl<K, S, V> extends AbstractStream<K, V> implements KTable<
|
|||
valueSerde = materializedInternal.valueSerde();
|
||||
queryableStoreName = materializedInternal.queryableStoreName();
|
||||
// only materialize if materialized is specified and it has queryable name
|
||||
final StoreFactory storeFactory = queryableStoreName != null ? (new KeyValueStoreMaterializer<>(materializedInternal)) : null;
|
||||
storeBuilder = Collections.singleton(new FactoryWrappingStoreBuilder<>(storeFactory));
|
||||
if (queryableStoreName != null) {
|
||||
final StoreFactory storeFactory = new KeyValueStoreMaterializer<>(materializedInternal);
|
||||
storeBuilder = Collections.singleton(new FactoryWrappingStoreBuilder<>(storeFactory));
|
||||
} else {
|
||||
storeBuilder = null;
|
||||
}
|
||||
} else {
|
||||
keySerde = this.keySerde;
|
||||
valueSerde = null;
|
||||
|
|
|
@ -436,6 +436,32 @@ public class KTableTransformValuesTest {
|
|||
new KeyValueTimestamp<>("A", "3", 15))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldCalculateCorrectOldValuesIfNotStatefulEvenNotMaterializedNoQueryableName() {
|
||||
builder
|
||||
.table(INPUT_TOPIC, CONSUMED)
|
||||
.transformValues(new StatelessTransformerSupplier(),
|
||||
Materialized.with(Serdes.String(), Serdes.Integer())
|
||||
)
|
||||
.groupBy(toForceSendingOfOldValues(), Grouped.with(Serdes.String(), Serdes.Integer()))
|
||||
.reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR)
|
||||
.mapValues(mapBackToStrings())
|
||||
.toStream()
|
||||
.process(capture);
|
||||
|
||||
driver = new TopologyTestDriver(builder.build(), props());
|
||||
final TestInputTopic<String, String> inputTopic =
|
||||
driver.createInputTopic(INPUT_TOPIC, new StringSerializer(), new StringSerializer());
|
||||
|
||||
inputTopic.pipeInput("A", "a", 5L);
|
||||
inputTopic.pipeInput("A", "aa", 15L);
|
||||
inputTopic.pipeInput("A", "aaa", 10);
|
||||
|
||||
assertThat(output(), equalTo(Arrays.asList(new KeyValueTimestamp<>("A", "1", 5),
|
||||
new KeyValueTimestamp<>("A", "2", 15),
|
||||
new KeyValueTimestamp<>("A", "3", 15))));
|
||||
}
|
||||
|
||||
private ArrayList<KeyValueTimestamp<String, String>> output() {
|
||||
return capture.capturedProcessors(1).get(0).processed();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue