Class JdbcXaSinkFunction<T>

  • All Implemented Interfaces:
    Serializable, AutoCloseable, org.apache.flink.api.common.functions.Function, org.apache.flink.api.common.functions.RichFunction, org.apache.flink.api.common.state.CheckpointListener, org.apache.flink.api.java.typeutils.InputTypeConfigurable, org.apache.flink.streaming.api.checkpoint.CheckpointedFunction, org.apache.flink.streaming.api.functions.sink.SinkFunction<T>

    @Internal
    public class JdbcXaSinkFunction<T>
    extends org.apache.flink.api.common.functions.AbstractRichFunction
    implements org.apache.flink.streaming.api.checkpoint.CheckpointedFunction, org.apache.flink.api.common.state.CheckpointListener, org.apache.flink.streaming.api.functions.sink.SinkFunction<T>, AutoCloseable, org.apache.flink.api.java.typeutils.InputTypeConfigurable
    JDBC sink function that uses XA transactions to provide exactly once guarantees. That is, if a checkpoint succeeds then all records emitted during it are committed in the database, and rolled back otherwise.

    Each parallel subtask has it's own transactions, independent from other subtasks. Therefore, consistency is only guaranteed within partitions.

    XA uses a two-phase commit protocol, which solves the consistency problem, but leaves the following issues:

    1. transactions may be abandoned, holding resources (e.g. locks, versions of rows)
    2. abandoned transactions collide with the new transactions if their IDs repeat after recovery
    3. commit requests may be repeated after job recovery, resulting in error responses and job failure

    The following table summarizes effects of failures during transaction state transitions and ways to mitigate them:

    Transition Methods What happens if transition lost Ways to mitigate
    none > started, started > ended open(), snapshotState() Database eventually discards these transactions
    1. Use globally unique XIDs
    2. derive XID from: checkpoint id, subtask id, "job id", "run id" (see SemanticXidGenerator).
    ended > prepared snapshotState() Database keeps these transactions prepared forever ("in-doubt" state)
    1. store ended transactions in state; rollback on job recovery (still doesn't cover all scenarios)
    2. call xa_recover() and xa_rollback() on job recovery; disabled by default in order not to affect transactions of other subtasks and apps
    3. setting transaction timeouts (not supported by most databases)
    4. manual recovery and rollback
    prepared > committed open(), notifyCheckpointComplete() Upon job recovery state contains committed transactions; or JM may notifyCheckpointComplete again after recovery.

    Committing results in XAER_NOTA error.

    Distinguish between transactions created during this run and restored from state and ignore XAER_NOTA for the latter.
    Since:
    1.13
    See Also:
    Serialized Form
    • Method Detail

      • initializeState

        public void initializeState​(org.apache.flink.runtime.state.FunctionInitializationContext context)
                             throws Exception
        Specified by:
        initializeState in interface org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
        Throws:
        Exception
      • open

        public void open​(org.apache.flink.configuration.Configuration configuration)
                  throws Exception
        Specified by:
        open in interface org.apache.flink.api.common.functions.RichFunction
        Overrides:
        open in class org.apache.flink.api.common.functions.AbstractRichFunction
        Throws:
        Exception
      • snapshotState

        public void snapshotState​(org.apache.flink.runtime.state.FunctionSnapshotContext context)
                           throws Exception
        Specified by:
        snapshotState in interface org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
        Throws:
        Exception
      • notifyCheckpointComplete

        public void notifyCheckpointComplete​(long checkpointId)
        Specified by:
        notifyCheckpointComplete in interface org.apache.flink.api.common.state.CheckpointListener
      • invoke

        public void invoke​(T value,
                           org.apache.flink.streaming.api.functions.sink.SinkFunction.Context context)
                    throws IOException
        Specified by:
        invoke in interface org.apache.flink.streaming.api.functions.sink.SinkFunction<T>
        Throws:
        IOException
      • close

        public void close()
                   throws Exception
        Specified by:
        close in interface AutoCloseable
        Specified by:
        close in interface org.apache.flink.api.common.functions.RichFunction
        Overrides:
        close in class org.apache.flink.api.common.functions.AbstractRichFunction
        Throws:
        Exception
      • setInputType

        public void setInputType​(org.apache.flink.api.common.typeinfo.TypeInformation<?> type,
                                 org.apache.flink.api.common.ExecutionConfig executionConfig)
        Specified by:
        setInputType in interface org.apache.flink.api.java.typeutils.InputTypeConfigurable