Closed fqtab closed 5 months ago
@tabmatfournier this is just a draft to illustrate the idea but feel free to ask any questions here. I don't intend to merge this; I assume you'll pull in any changes back into your PR if we go with this approach.
Thanks. I'll take a look
Not sure this is much of an improvement TBH. Very broad net.
The real problem is you have a bunch of errors in the writer .write
method which happen long before you write a record to iceberg:
Actually writing (to an appender) is not the issue. But the above logic is split between the WriterFactory and the writer itself.
Might be worth pulling that out into distinct steps (catalogAPI does this for the first two cases, may be worth introducing a RecordConverter outside of the writer). It might make the wrapping you want more clear.
< E.g. if a user wants a dead-letter-table per topic, users can just write a WriteExceptionHandler that sends bads record from different topics to different tables.
This is nice but you also have to handle cases where you don't know the table it's going to (because the table name itself is invalid and the catalog throws) when trying to create it. It's not that simple.
7. Relatively straight-forward change to the codebase
disagree, and there is functionality missing that once added ... will make this more complicated.
Current plan:
- Converter/SMT exceptions (i.e. things before
SinkTask.put
), users should configure the connector iniceberg.tables.dynamic-enabled
with aiceberg.tables.route-field
and write an exception-handling-SMT that points to the the dead-letter-table.
I don't want to gate keep this feature only for users who use dynamic-enabled. There is nothing here restricting other users from using this other than "simplifying code" (of course it's simplifying code --we are ignoring valid configurations).
I believe this to be a deal breaker.
Similarly, we (maintainers) don't have to worry about adding exception handling code to the connector
- I'm particularly worried about REST catalogs which only really have a REST spec. There is no guarantee an exception thrown by one REST Catalog is non-transient versus another.
If that is the case, why are we providing any implementations at all. IMHO we can't get away with just providing the interface for users. We have to provide some amount of default implementation.
- Users can have as many or as few dead letter tables as they want, just write an appropriate handler to do it
Is this something we want?
Remember: you will still need a default dead letter table because there are many cases where you don't know where to send the record to (error transform failures are a good example of this), so you are left to config value/fn of connector name. I would argue topic name is a poor choice but you could do that if you wanted. It is a poor choice because the user may be running several connectors for the same topic and using different SMTs.
I'm not convinced you can just "write an appropriate handler to do it" because you often won't have any context for where to route to.
Not sure this is much of an improvement TBH. Very broad net.
The real problem is you have a bunch of errors in the writer
.write
method which happen long before you write a record to iceberg:
- a laundry list of issues creating a table (table names, partition specs, etc) including values from hardcoded configs
- partition evolution cases
- iceberg record conversion issues
Actually writing (to an appender) is not the issue. But the above logic is split between the WriterFactory and the writer itself.
Might be worth pulling that out into distinct steps (catalogAPI does this for the first two cases, may be worth introducing a RecordConverter outside of the writer). It might make the wrapping you want more clear.
Oh my bad, i threw this together pretty quickly. I thought most of the errors would only happen when we write SinkRecords to files but if there are errors we think it would be worth catching higher up (e.g. when creating a Writer), then we can certainly move the try {} catch ()
to a higher level, not opposed to that.
- Converter/SMT exceptions (i.e. things before
SinkTask.put
), users should configure the connector iniceberg.tables.dynamic-enabled
with aiceberg.tables.route-field
and write an exception-handling-SMT that points to the the dead-letter-table.I don't want to gate keep this feature only for users who use dynamic-enabled. There is nothing here restricting other users from using this other than "simplifying code" (of course it's simplifying code --we are ignoring valid configurations).
I believe this to be a deal breaker.
No problems, there are things we can do to mitigate this. Alex had a pretty good idea about allowing dynamic mode to fallback to static mapping if the route field is missing/null. I see that as a valuable feature for usecases outside of handling converter/SMT failures so I would be in favour of supporting that change to the connector code. That should address this concern entirely unless I'm missing something else.
there are errors we think it would be worth catching higher up (e.g. when creating a Writer), then we can certainly move the
try {} catch ()
to a higher level, not opposed to that.
It's not that easy. Unfortunately where you need to catch is a shotgun (worker, writer, writer factory, schemautils, etc.)
- Converter/SMT exceptions (i.e. things before
SinkTask.put
), users should configure the connector iniceberg.tables.dynamic-enabled
with aiceberg.tables.route-field
and write an exception-handling-SMT that points to the the dead-letter-table.I don't want to gate keep this feature only for users who use dynamic-enabled. There is nothing here restricting other users from using this other than "simplifying code" (of course it's simplifying code --we are ignoring valid configurations). I believe this to be a deal breaker.
No problems, there are things we can do to mitigate this. Alex had a pretty good idea about allowing dynamic mode to fallback to static mapping if the route field is missing/null. I see that as a valuable feature for usecases outside of handling converter/SMT failures so I would be in favour of supporting that change to the connector code. That should address this concern entirely unless I'm missing something else.
Not opposed but I can show you how this is tricky/challenging in a screenshare.
If that is the case, why are we providing any implementations at all. IMHO we can't get away with just providing the interface for users. We have to provide some amount of default implementation.
Absolutely. That's why your PR is offering an ExceptionHandlingTransform. The only additional thing this PR is requiring is also an appropriate WriteExceptionHandler. We need to provide a good default implementation of that. And we can do that correctly for Tabular's REST Catalog with a TabularWriteExceptionHandler for example (+ Hive + anything else, up to us). For other REST Catalogs? We can't offer anything of the sort without seeing their code.
there are errors we think it would be worth catching higher up (e.g. when creating a Writer), then we can certainly move the
try {} catch ()
to a higher level, not opposed to that.It's not that easy. Unfortunately where you need to catch is a shotgun (worker, writer, writer factory, schemautils, etc.)
~Sure, let's pair later, sounds like I might have dodged a lot of the complexity by focusing on only the write path :D~
Scratch that, this should cover all errors coming from WriterFactory, IcebergWriter, and SchemaUtills as well? https://github.com/tabular-io/iceberg-kafka-connect/blob/925dbcb74627a9c53370db55ec8967677c5605a2/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/MultiTableWriter.java#L49
Adds a configuration option to specify a
WriteExceptionHandler
to use to handle any exceptions while writing SinkRecords to files.I haven't tested any of this yet :D but in theory:
SinkTask.put
would be captured by the user configuredWriteExceptionHandler
and handled there.SinkTask.put
), users should configure the connector iniceberg.tables.dynamic-enabled
with aiceberg.tables.route-field
and write an exception-handling-SMT that points to the the dead-letter-table.WriteExceptionHandler
and anExceptionHandlingSMT
that should work for 90% of usecases.Pros
byte[]
, you canWriteExceptionHandler
that sends bads record from different topics to different tables.SinkTaskContext.errantRecordReporter.report
APIWriteExceptionHandler
implementation)Cons
WriteExceptionHandler
implementationWriteExceptionHanlder
implementation, similar to how we provide sample SMT implementations alreadyWriteExceptionHandler
implementation is available on the classpathWriteExceptionHandler
implementations in the kafka-connect-iceberg project (but we should get some stable implementations first)How to
It might be a little hard to see how people could use this feature for specific use cases so I've prepared a separate draft PR to show this: https://github.com/tabular-io/iceberg-kafka-connect/pull/244