Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 7F404200C50 for ; Fri, 24 Mar 2017 19:28:51 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 7DEC0160B9A; Fri, 24 Mar 2017 18:28:51 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id A438E160B93 for ; Fri, 24 Mar 2017 19:28:50 +0100 (CET) Received: (qmail 73048 invoked by uid 500); 24 Mar 2017 18:28:49 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 73021 invoked by uid 99); 24 Mar 2017 18:28:49 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 24 Mar 2017 18:28:49 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id C86D1C0E5C for ; Fri, 24 Mar 2017 18:28:47 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.201 X-Spam-Level: X-Spam-Status: No, score=-99.201 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id seP4cRssrwMC for ; Fri, 24 Mar 2017 18:28:46 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 056345FE0B for ; Fri, 24 Mar 2017 18:28:46 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 3FFA1E0BFB for ; Fri, 24 Mar 2017 18:28:43 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 3081924076 for ; Fri, 24 Mar 2017 18:28:42 +0000 (UTC) Date: Fri, 24 Mar 2017 18:28:42 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-3257) Add Exactly-Once Processing Guarantees in Iterative DataStream Jobs MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 24 Mar 2017 18:28:51 -0000 [ https://issues.apache.org/jira/browse/FLINK-3257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15940875#comment-15940875 ] ASF GitHub Bot commented on FLINK-3257: --------------------------------------- Github user gyfora commented on a diff in the pull request: https://github.com/apache/flink/pull/1668#discussion_r107968567 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java --- @@ -119,13 +183,154 @@ protected void cleanup() throws Exception { * Creates the identification string with which head and tail task find the shared blocking * queue for the back channel. The identification string is unique per parallel head/tail pair * per iteration per job. - * - * @param jid The job ID. - * @param iterationID The id of the iteration in the job. + * + * @param jid The job ID. + * @param iterationID The id of the iteration in the job. * @param subtaskIndex The parallel subtask number * @return The identification string. */ public static String createBrokerIdString(JobID jid, String iterationID, int subtaskIndex) { return jid + "-" + iterationID + "-" + subtaskIndex; } + + /** + * An internal operator that solely serves as a state logging facility for persisting, + * partitioning and restoring output logs for dataflow cycles consistently. To support concurrency, + * logs are being sliced proportionally to the number of concurrent snapshots. This allows committed + * output logs to be uniquely identified and cleared after each complete checkpoint. + *

+ * The design is based on the following assumptions: + *

+ * - A slice is named after a checkpoint ID. Checkpoint IDs are numerically ordered within an execution. + * - Each checkpoint barrier arrives back in FIFO order, thus we discard log slices in respective FIFO order. + * - Upon restoration the logger sorts sliced logs in the same FIFO order and returns an Iterable that + * gives a singular view of the log. + *

+ * TODO it seems that ListState.clear does not unregister state. We need to put a hook for that. + * + * @param + */ + public static class UpstreamLogger extends AbstractStreamOperator implements OneInputStreamOperator { + + private final StreamConfig config; + + private LinkedList>> slicedLog = new LinkedList<>(); + + private UpstreamLogger(StreamConfig config) { + this.config = config; + } + + public void logRecord(StreamRecord record) throws Exception { + if (!slicedLog.isEmpty()) { + slicedLog.getLast().add(record); + } + } + + public void createSlice(String sliceID) throws Exception { + ListState> nextSlice = + getOperatorStateBackend().getOperatorState(new ListStateDescriptor<>(sliceID, + config.>getTypeSerializerOut(getUserCodeClassloader()))); + slicedLog.addLast(nextSlice); + } + + public void discardSlice() { + ListState> logToEvict = slicedLog.pollFirst(); + logToEvict.clear(); + } + + public Iterable> getReplayLog() throws Exception { + final List logSlices = new ArrayList<>(getOperatorStateBackend().getRegisteredStateNames()); + Collections.sort(logSlices, new Comparator() { + @Override + public int compare(String o1, String o2) { + return Long.valueOf(o1).compareTo(Long.valueOf(o2)); + } + }); + + final List>> wrappedIterators = new ArrayList<>(); + for (String splitID : logSlices) { + wrappedIterators.add(getOperatorStateBackend() + .getOperatorState(new ListStateDescriptor<>(splitID, + config.>getTypeSerializerOut(getUserCodeClassloader()))).get().iterator()); + } + + if (wrappedIterators.size() == 0) { + return new Iterable>() { + @Override + public Iterator> iterator() { + return Collections.emptyListIterator(); + } + }; + } + + return new Iterable>() { + @Override + public Iterator> iterator() { + + return new Iterator>() { + int indx = 0; + Iterator> currentIterator = wrappedIterators.get(0); + + @Override + public boolean hasNext() { + if (!currentIterator.hasNext()) { + progressLog(); + } + return currentIterator.hasNext(); + } + + @Override + public StreamRecord next() { + if (!currentIterator.hasNext() && indx < wrappedIterators.size()) { + progressLog(); + } + return currentIterator.next(); + } + + private void progressLog() { + while (!currentIterator.hasNext() && ++indx < wrappedIterators.size()) { + currentIterator = wrappedIterators.get(indx); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + }; + } + }; + } + + public void clearLog() throws Exception { + for (String outputLogs : getOperatorStateBackend().getRegisteredStateNames()) { --- End diff -- It's kind of bad that we can't remove the state completely and keep iterating over them when replaying the log... > Add Exactly-Once Processing Guarantees in Iterative DataStream Jobs > ------------------------------------------------------------------- > > Key: FLINK-3257 > URL: https://issues.apache.org/jira/browse/FLINK-3257 > Project: Flink > Issue Type: Improvement > Components: DataStream API > Reporter: Paris Carbone > Assignee: Paris Carbone > > The current snapshotting algorithm cannot support cycles in the execution graph. An alternative scheme can potentially include records in-transit through the back-edges of a cyclic execution graph (ABS [1]) to achieve the same guarantees. > One straightforward implementation of ABS for cyclic graphs can work as follows along the lines: > 1) Upon triggering a barrier in an IterationHead from the TaskManager start block output and start upstream backup of all records forwarded from the respective IterationSink. > 2) The IterationSink should eventually forward the current snapshotting epoch barrier to the IterationSource. > 3) Upon receiving a barrier from the IterationSink, the IterationSource should finalize the snapshot, unblock its output and emit all records in-transit in FIFO order and continue the usual execution. > -- > Upon restart the IterationSource should emit all records from the injected snapshot first and then continue its usual execution. > Several optimisations and slight variations can be potentially achieved but this can be the initial implementation take. > [1] http://arxiv.org/abs/1506.08603 -- This message was sent by Atlassian JIRA (v6.3.15#6346)