[
https://issues.apache.org/jira/browse/FLINK-9428?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16488678#comment-16488678
]
ASF GitHub Bot commented on FLINK-9428:
---------------------------------------
Github user aljoscha commented on a diff in the pull request:
https://github.com/apache/flink/pull/6066#discussion_r190512162
--- Diff:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OperatorChainTest.java
---
@@ -0,0 +1,142 @@
+/*
+ * 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.runtime.tasks;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.operators.testutils.MockEnvironment;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
+import
org.apache.flink.streaming.runtime.operators.StreamOperatorChainingTest;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import
org.apache.flink.streaming.runtime.streamstatus.StreamStatusProvider;
+import
org.apache.flink.streaming.runtime.tasks.OperatorChain.BroadcastingOutputCollector;
+import
org.apache.flink.streaming.runtime.tasks.OperatorChain.ChainingOutput;
+import
org.apache.flink.streaming.runtime.tasks.OperatorChain.WatermarkGaugeExposingOutput;
+
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+
+/**
+ * This class test the {@link OperatorChain}.
+ *
+ * <p>It takes a different (simpler) approach at testing the operator
chain than
+ * {@link StreamOperatorChainingTest}.
+ */
+public class OperatorChainTest {
+
+ @Test
+ public void testPrepareCheckpointPreBarrier() throws Exception {
+ final AtomicInteger intRef = new AtomicInteger();
+
+ final OneInputStreamOperator<String, String> one = new
ValidatingOperator(intRef, 0);
+ final OneInputStreamOperator<String, String> two = new
ValidatingOperator(intRef, 1);
+ final OneInputStreamOperator<String, String> three = new
ValidatingOperator(intRef, 2);
+
+ final OperatorChain<?, ?> chain = setupOperatorChain(one, two,
three);
+
chain.prepareSnapshotPreBarrier(ValidatingOperator.CHECKPOINT_ID);
+
+ assertEquals(3, intRef.get());
+ }
+
+ //
------------------------------------------------------------------------
+ // Operator Chain Setup Utils
+ //
------------------------------------------------------------------------
+
+ @SafeVarargs
+ private static <T, OP extends StreamOperator<T>> OperatorChain<T, OP>
setupOperatorChain(
--- End diff --
this is maybe a bit much mocking
> Allow operators to flush data on checkpoint pre-barrier
> -------------------------------------------------------
>
> Key: FLINK-9428
> URL: https://issues.apache.org/jira/browse/FLINK-9428
> Project: Flink
> Issue Type: New Feature
> Components: State Backends, Checkpointing
> Reporter: Stephan Ewen
> Assignee: Stephan Ewen
> Priority: Major
> Fix For: 1.6.0
>
>
> Some operators maintain some small transient state that may be inefficient to
> checkpoint, especially when it would need to be checkpointed also in a
> re-scalable way.
> An example are opportunistic pre-aggregation operators, which have small the
> pre-aggregation state that is frequently flushed downstream.
> Rather that persisting that state in a checkpoint, it can make sense to flush
> the data downstream upon a checkpoint, to let it be part of the downstream
> operator's state.
> This feature is sensitive, because flushing state has a clean implication on
> the downstream operator's checkpoint alignment. However, used with care, and
> with the new back-pressure-based checkpoint alignment, this feature can be
> very useful.
> Because it is sensitive, I suggest to make this only an internal feature
> (accessible to operators) and NOT expose it in the public API at this point.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)