You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tzulitai <gi...@git.apache.org> on 2017/08/03 07:31:02 UTC

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

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

    https://github.com/apache/flink/pull/4368#discussion_r131070533
  
    --- 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 --
    
    I think this has to be `ListState<Tuple2<Long, TXN>>` or the original `ListState<TransactionAndCheckpoint<TXN>>`.
    
    Using a map instead of a list for `pendingCommitTransactions ` is ok for bookkeeping, but when snapshotting this map we need to make sure the snapshotted transactions are state elements that can be redistributable independent of each other. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---