GitHub user StephanEwen opened a pull request:

    https://github.com/apache/flink/pull/1163

    [FLINK-2727] [streaming] Add a base class for Message Queue Sources that 
acknowledge messages by ID

    Several message queues (RabbitMQ, Amazon SQS) have the pattern that you 
retrieve messages and acknowledge them back by ID. This pull request adds a 
simple base non-parallel source that provides tooling for:
    
      - Collecting the IDs of elements emitted between two checkpoints
      - Persisting them with the checkpoint, respecting proper serialization
      - Acknowledging them when a checkpoint is notified of completion.
    
    This assumes that the Message Queues retain unacknowledged messages and 
re-emit them after the acknowledgement period expired.
    
    ### Form the class header
    
    The mechanism for this source assumes that messages are identified by a 
unique ID.
    When messages are taken from the message queue, the message must not be 
dropped immediately, but must be retained until acknowledged. Messages that are 
not acknowledged within a certain time interval will be served again (to a 
different connection, established by the recovered source).
    
    Note that this source can give no guarantees about message order in the 
case of failures, because messages that were retrieved but not yet acknowledged 
will be returned later again, after a set of messages that was not retrieved 
before the failure.
    
    Internally, this source gathers the IDs of elements it emits. Per 
checkpoint, the IDs are stored and acknowledged when the checkpoint is 
complete. That way, no message is acknowledged unless it is certain that it has 
been successfully processed throughout the topology and the updates to any 
state caused by that message are persistent.
    
    All messages that are emitted and successfully processed by the streaming 
program will eventually be acknowledged. In corner cases, the source may 
acknowledge certain IDs multiple times, if a failure occurs while acknowledging.
    
    A typical way to use this base in a source function is by implementing a 
run() method as follows:
    ```java
    public void run(SourceContext<Type> ctx) throws Exception {
        while (running) {
            Message msg = queue.retrieve();
            synchronized (ctx.getCheckpointLock()) {
                ctx.collect(msg.getMessageData());
                addId(msg.getMessageId());
            }
        }
    }
    ```

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/StephanEwen/incubator-flink messagequeuesource

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1163.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1163
    
----
commit 505bd0baa34560ca8a7f2744b3b7890152133a1e
Author: Stephan Ewen <[email protected]>
Date:   2015-09-22T11:23:56Z

    [FLINK-2727] [streaming] Add a base class for Message Queue Sources that 
acknowledge messages by ID.

----


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to