[
https://issues.apache.org/jira/browse/KAFKA-10179?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17139926#comment-17139926
]
Rohan Desai commented on KAFKA-10179:
-------------------------------------
I'm not sure it's correct to use the same "topic" name for materializing
optimized source tables, as it's logically different data. In the normal flow
(not recovery), we're taking the topic data, validating/transforming it by
deserializing it (which might apply some transforms like projecting just fields
of interest), and then serializing it, and then writing it into the store. So
the "topic" we pass to the serializer should be different since it represents
different data from the source topic.
This has consequences in practice when used with a schema registry using the
confluent serializers. If we use the same topic, `serialize` might register a
different schema with the source subject, which we probably don't want.
I think the technically correct thing to do (though this is of course more
expensive) would be (when the source table is optimized) to deserialize and
serialize each record when restoring.
Another issue that I think exists (need to try to reproduce) that
deserializing/serializing would solve is skipped validation. The source topic
deserializer functions as a sort of validator for records from the source
topic. When the streams app is configured to skip on deserialization errors,
bad source records are just skipped. However if we restore by just writing
those records to the state store, we now hit the deserialization error when
reading the state store, which is a query-killing error.
> State Store Passes Wrong Changelog Topic to Serde for Optimized Source Tables
> -----------------------------------------------------------------------------
>
> Key: KAFKA-10179
> URL: https://issues.apache.org/jira/browse/KAFKA-10179
> Project: Kafka
> Issue Type: Bug
> Components: streams
> Affects Versions: 2.5.0
> Reporter: Bruno Cadonna
> Assignee: Bruno Cadonna
> Priority: Major
> Fix For: 2.7.0
>
>
> {{MeteredKeyValueStore}} passes the name of the changelog topic of the state
> store to the state store serdes. Currently, it always passes {{<application
> ID>-<store name>-changelog}} as the changelog topic name. However, for
> optimized source tables the changelog topic is the source topic.
> Most serdes do not use the topic name passed to them. However, if the serdes
> actually use the topic name for (de)serialization, e.g., when Kafka Streams
> is used with Confluent's Schema Registry, a
> {{org.apache.kafka.common.errors.SerializationException}} is thrown.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)