CrynetLogistics commented on a change in pull request #18449:
URL: https://github.com/apache/flink/pull/18449#discussion_r790586157



##########
File path: 
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/AsyncSinkBaseBuilder.java
##########
@@ -37,6 +38,7 @@
         ConcreteBuilderT extends AsyncSinkBaseBuilder<?, ?, ?>> {
 
     private ElementConverter<InputT, RequestEntryT> elementConverter;
+    private FatalExceptionHandler<RequestEntryT> fatalExceptionHandler;

Review comment:
       Hi @nirtsruya thanks for your contribution.
   
   I see the following options:
   
   1. Keeping a mapping between `InputT` and `RequestEntryT` in the 
`SinkWriter` and only exposing the `InputT`.
   Advantages:
    - Only exposes the `InputT` to the user (would not break encapsulation, 
underlying sink implementation free to change)
   
   Disadvantages:
    - Sink would require ~2x the memory to store the corresponding `InputT`s
    - Would complicate `SinkWriter` implementation
   
   2. Return a `toString`-style printout of the failed `RequestEntryT`s - since 
most people really only want to append a representative list of failed entries 
in an appropriate format to their message log, we can provide a 
`shortFailedList` with a few elements (or under 200 chars) and a 
`fullFailedList`. The sink implementer would be in the best position to 
determine what is a 'representative list of failed entries in an appropriate 
format' for their particular set of `RequestEntryT`s.
   i.e.
   ```
   public interface FatalExceptionHandler<RequestEntryT> extends Serializable {
       void handle(String shortFailedList, String fullFailedList, Exception 
fatalException) throws Exception;
   ...
   ```
   Advantages:
    - Would not expose `RequestEntryT` to the user (would not break 
encapsulation, underlying sink implementation free to change)
   
   Disadvantages:
    - User not free to use the `RequestEntryT`s in whatever way they like - 
sort/concat/parse&determine error/etc.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to