Some more details: Changelog Topics: in the DSL, serialization happens in the store level (MeteredXxxStore) and hence any serialization exception should be thrown to the `Processor` before the message is written to the changelog topic or underlying store. When the actually write to the changelog topic happens, no serialization exception can occur any longer as we already have `<byte[],byte[]>` key-value-pairs at hand.
Note, that for the existing ProductionExceptionHandler, the handler might still swallow an exception currently and state store and changelog topic may diverge. If we believe that this is an issue we should fix, it would be in addition to allow handling serialization exceptions -- on the other hand, one could argue, that it's the users responsibility to check the topic name on the exception handler callback and don't swallow for changelog topics. If we follow this argument, we don't need to extend the scope of the KIP but push this onto the users (more flexibility == more responsibility). Getting back to the DSL serialization exception handler, there is still the problem that users don't have any chance to catch such an exception atm. For PAPI users, this issue does not persist as they can add try-catch blocks to react to serialization exceptions and customize their code accordingly. Similar for EOS, users can register a handler, but it would of course violate the guarantees if they swallow an exception. Again, we could disallow it, or argue that it's a users own responsibility. To move this KIP forward, maybe we can just (0) add the handler for serialization exceptions when writing into any topic and consider it an incremental improvement. Ie, (1) we keep the door open to let state and changelog topic diverge (current status) (2) we allow people to violate EOS (current state) (3) and we don't improve the handling of DSL state store serialization exceptions. We could address (1), (2), and/or (3) in follow up KIPs. Thoughts? Let us know if you only want to address (0), or extend the current KIP to include any of (1-3). Would love the hear from others, too. -Matthias On 10/17/19 2:52 PM, Walker Carlson wrote: > I have read through the logs and I am of the opinion that while we should > not let the store and the change Log diverge. However, it is not obvious > how we would be able to do that and allow the custom serializer to effect > that topic. In order to get around this we can create a shell around the > handler that will not trigger the custom handler if it is a changeLog > topic. This is also a problem that we can fix in the general handler, so I > think we can add that fix to this kip. > > As for the repartition topics I can not see any reason to not apply the > custom logic. By default it fails in any case and does not have to be > changed from that. > > For the DSL I think that that can come later as this change would not > affect the status quo. EOS and this handler can be made > mutually exclusive or we could give users the option to use both with a > warning. I would be interested in hearing other people's suggestions about > that.Walker > > On Tue, Oct 15, 2019 at 11:50 PM Matthias J. Sax <matth...@confluent.io> > wrote: > >> Walker, >> >> thanks for picking up this KIP. Did you read the previous discussion? >> It's still unclear if we want to apply the handler to repartition topics >> or not, and how errors for stores and changelog topic should be handled. >> For the Processor API, users could catch `SerializationException` but >> for the DSL this is not possible. Even if there is no serialization >> problem, it's questionable if we should allow to swallow error when >> writing into the changelog topic, because the store content and the >> topic content could diverge, what is especially critical for the EOS case. >> >> >> -Matthias >> >> On 10/15/19 10:25 AM, Walker Carlson wrote: >>> Hello all, >>> >>> I would like to restart the discussion of this KIP 399 >>> < >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-399%3A+Extend+ProductionExceptionHandler+to+cover+serialization+exceptions >>> . >>> I think it is some low hanging fruit that could be quite beneficial. >>> >>> Thanks, >>> Walker >>> >> >> >
signature.asc
Description: OpenPGP digital signature