Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4910#discussion_r147658988
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
    @@ -58,18 +61,37 @@
                extends RichSinkFunction<IN>
                implements CheckpointedFunction, CheckpointListener {
     
    -   private static final Logger LOG = 
LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +   private final Logger log;
     
    -   protected final ListStateDescriptor<State<TXN, CONTEXT>> 
stateDescriptor;
    +   private final Clock clock;
     
    -   protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = 
new LinkedHashMap<>();
    +   protected final LinkedHashMap<Long, TransactionHolder<TXN>> 
pendingCommitTransactions = new LinkedHashMap<>();
     
    -   @Nullable
    -   protected TXN currentTransaction;
        protected Optional<CONTEXT> userContext;
     
        protected ListState<State<TXN, CONTEXT>> state;
     
    +   private final ListStateDescriptor<State<TXN, CONTEXT>> stateDescriptor;
    +
    +   private TransactionHolder<TXN> currentTransaction;
    +
    +   /**
    +    * Specifies the maximum time a transaction should remain open.
    +    */
    +   private long transactionTimeout = Long.MAX_VALUE;
    +
    +   /**
    +    * If true, any exception thrown in {@link #recoverAndCommit(Object)} 
will be caught instead of
    +    * propagated.
    +    */
    +   private boolean failureOnCommitAfterTransactionTimeoutDisabled;
    +
    +   /**
    +    * If a transaction's elapsed time reaches this percentage of the 
transactionTimeout, a warning
    +    * message will be logged. Value must be in range [0,1]. Negative value 
disables warnings.
    +    */
    +   private double transactionTimeoutWarningRatio = -1;
    --- End diff --
    
    I would prefer to implement this sanity check in `FlinkKafkaProducer011`, 
since it's a walk around Kafka's bug and unlikely to be useful in general case. 
And after Kafka `0.11.0.2` or `1.0.0` release we won't need this code anymore.


---

Reply via email to