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

    https://github.com/apache/flink/pull/4368#discussion_r131330545
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
    @@ -51,39 +48,52 @@
      * @param <TXN> Transaction to store all of the information required to 
handle a transaction (must be Serializable)
      */
     @PublicEvolving
    -public abstract class TwoPhaseCommitSinkFunction<IN, TXN extends 
Serializable>
    +public abstract class TwoPhaseCommitSinkFunction<IN, TXN>
                extends RichSinkFunction<IN>
                implements CheckpointedFunction, CheckpointListener {
     
        private static final Logger LOG = 
LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
     
    -   protected final ListStateDescriptor<TransactionAndCheckpoint<TXN>> 
pendingCommitTransactionsDescriptor;
    +   protected final ListStateDescriptor<Map<Long, TXN>> 
pendingCommitTransactionsDescriptor;
        protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
     
    -   protected final List<TransactionAndCheckpoint<TXN>> 
pendingCommitTransactions = new ArrayList<>();
    +   protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = 
new LinkedHashMap<>();
     
        @Nullable
        protected TXN currentTransaction;
        protected ListState<TXN> pendingTransactionsState;
    -   protected ListState<TransactionAndCheckpoint<TXN>> 
pendingCommitTransactionsState;
    -
    -   public TwoPhaseCommitSinkFunction(Class<TXN> txnClass) {
    -           this(
    -                   TypeInformation.of(txnClass),
    -                   TypeInformation.of(new 
TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    -   }
    +   protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    --- End diff --
    
    Whether transactions can be redistributed depends on the system that we 
communicate with. For Kafka 0.11 I'm not sure. Now when I think about it a 
little bit more, I will change it to `ListState<List<Tuple2<Long, TXN>>>`, so 
that we can guarantee that transactions will be recovered in the same order 
they were created.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to