You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StephanEwen <gi...@git.apache.org> on 2015/09/22 13:33:28 UTC

[GitHub] flink pull request: [FLINK-2727] [streaming] Add a base class for ...

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

[GitHub] flink pull request: [FLINK-2727] [streaming] Add a base class for ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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.
---

[GitHub] flink pull request: [FLINK-2727] [streaming] Add a base class for ...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1163#issuecomment-144950240
  
    All right, merging this...


---
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.
---

[GitHub] flink pull request: [FLINK-2727] [streaming] Add a base class for ...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1163#discussion_r40933422
  
    --- Diff: flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java ---
    @@ -0,0 +1,173 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.api.functions.source;
    +
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
    +import org.apache.flink.streaming.api.checkpoint.Checkpointed;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.state.SerializedCheckpointData;
    +
    +import java.util.ArrayDeque;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +/**
    + * Abstract base class for data sources that receive elements from a message queue and
    + * acknowledge them back by IDs.
    + * <p>
    + * 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).
    + * <p>
    + * Note that this source can give no guarantees about message order in teh case of failures,
    --- End diff --
    
    Typo: teh


---
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.
---

[GitHub] flink pull request: [FLINK-2727] [streaming] Add a base class for ...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1163#issuecomment-144015953
  
    Any objection against merging this?


---
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.
---

[GitHub] flink pull request: [FLINK-2727] [streaming] Add a base class for ...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the pull request:

    https://github.com/apache/flink/pull/1163#issuecomment-144777018
  
    Valuable base class for integrating messaging queuing systems. Looks good.


---
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.
---