From de674f19fcbe9955cb4208ef0938fe5b0f7adc90 Mon Sep 17 00:00:00 2001 From: Chen Qin Date: Fri, 21 Oct 2016 12:38:04 -0700 Subject: [PATCH 1/5] allow mutpile output stream --- .../flink/api/common/typeinfo/OutputTag.java | 35 +++++++++++ .../flink/api/common/typeinfo/TypeHint.java | 2 +- .../api/common/typeutils/OutputTagUtil.java | 31 ++++++++++ .../org/apache/flink/util/RichCollector.java | 30 +++++++++ .../examples/wordcount/WordCount.java | 8 +++ .../streaming/api/datastream/DataStream.java | 6 +- .../SingleOutputStreamOperator.java | 9 +++ .../streaming/api/graph/StreamConfig.java | 2 + .../flink/streaming/api/graph/StreamEdge.java | 9 ++- .../streaming/api/graph/StreamGraph.java | 3 +- .../api/graph/StreamGraphGenerator.java | 46 ++++++++++---- .../flink/streaming/api/graph/StreamNode.java | 14 +++++ .../api/operators/TimestampedCollector.java | 13 +++- .../SideOutputTransformation.java | 62 +++++++++++++++++++ .../runtime/io/RecordWriterOutput.java | 13 ++-- .../runtime/streamrecord/StreamRecord.java | 12 ++++ .../runtime/tasks/OperatorChain.java | 30 +++++---- .../runtime/tasks/OneInputStreamTaskTest.java | 1 + .../runtime/tasks/StreamTaskTestHarness.java | 2 +- .../tasks/TwoInputStreamTaskTestHarness.java | 6 +- 20 files changed, 291 insertions(+), 43 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java create mode 100644 flink-core/src/main/java/org/apache/flink/util/RichCollector.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java new file mode 100644 index 0000000000000..6e5ea0195f74b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java @@ -0,0 +1,35 @@ +/* + * 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.api.common.typeinfo; + +import java.io.Serializable; + +/** + * Created by chenqin on 10/20/16. + */ +public abstract class OutputTag extends TypeHint implements Serializable{ + + @Override + public boolean equals(Object obj) { + return super.equals(obj); + } + + public void setTypeInfo(TypeInformation t) { + this.typeInfo = t; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java index 975d6e36942f2..b84935a62c147 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java @@ -38,7 +38,7 @@ public abstract class TypeHint { /** The type information described by the hint */ - private final TypeInformation typeInfo; + protected TypeInformation typeInfo; /** * Creates a hint for the generic type in the class signature. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java new file mode 100644 index 0000000000000..375c917743d0b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java @@ -0,0 +1,31 @@ +package org.apache.flink.api.common.typeutils; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.List; +import org.apache.flink.api.common.typeinfo.OutputTag; +import org.apache.flink.api.common.typeinfo.TypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.api.java.typeutils.TypeInfoParser; +import org.codehaus.jackson.map.type.TypeParser; + + +/** + * Created by chenqin on 10/21/16. + */ +public class OutputTagUtil { + public static List getOutputTagName(OutputTag tag){ + return Arrays.asList(tag.getTypeInfo().toString()); + } + + public static TypeInformation getSideOutputTypeInfo(List names){ + //HACK Alert + if(names.size() == 1 && names.get(0) == "String") { + return TypeInfoParser.parse(names.get(0)); + } else { + return null; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/util/RichCollector.java b/flink-core/src/main/java/org/apache/flink/util/RichCollector.java new file mode 100644 index 0000000000000..476a8b7d3bfa5 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/RichCollector.java @@ -0,0 +1,30 @@ +/* + * 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.util; + +import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.typeinfo.OutputTag; + +/** + * support collect with outputtag + */ +@Public +public interface RichCollector extends Collector{ + void collect(OutputTag tag, S value); +} diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java index 4290878dab02d..160eede589465 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java @@ -18,12 +18,14 @@ package org.apache.flink.streaming.examples.wordcount; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; +import org.apache.flink.util.RichCollector; /** * Implements the "WordCount" program that computes a simple word occurrence @@ -81,6 +83,8 @@ public static void main(String[] args) throws Exception { // group by the tuple field "0" and sum up tuple field "1" .keyBy(0).sum(1); + text.flatMap(new Tokenizer()).getOutput(sideOut).print(); + // emit result if (params.has("output")) { counts.writeAsText(params.get("output")); @@ -93,6 +97,8 @@ public static void main(String[] args) throws Exception { env.execute("Streaming WordCount"); } + static final OutputTag sideOut = new OutputTag() {}; + // ************************************************************************* // USER FUNCTIONS // ************************************************************************* @@ -118,6 +124,8 @@ public void flatMap(String value, Collector> out) out.collect(new Tuple2(token, 1)); } } + RichCollector> sideout = (RichCollector>) out; + sideout.collect(sideOut, "sideout"); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 204557db0100a..0231e7b855163 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.Utils; @@ -62,10 +63,7 @@ import org.apache.flink.streaming.api.operators.StreamFlatMap; import org.apache.flink.streaming.api.operators.StreamMap; import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.api.transformations.OneInputTransformation; -import org.apache.flink.streaming.api.transformations.PartitionTransformation; -import org.apache.flink.streaming.api.transformations.StreamTransformation; -import org.apache.flink.streaming.api.transformations.UnionTransformation; +import org.apache.flink.streaming.api.transformations.*; import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows; import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java index 614f19b42aafc..0f430184a4d7c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java @@ -17,15 +17,19 @@ package org.apache.flink.streaming.api.datastream; +import com.google.common.collect.Lists; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.functions.InvalidTypesException; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.api.transformations.SelectTransformation; +import org.apache.flink.streaming.api.transformations.SideOutputTransformation; import org.apache.flink.streaming.api.transformations.StreamTransformation; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.util.Preconditions; @@ -341,4 +345,9 @@ public SingleOutputStreamOperator slotSharingGroup(String slotSharingGroup) { transformation.setSlotSharingGroup(slotSharingGroup); return this; } + + public DataStream getOutput(OutputTag tag){ + SideOutputTransformation sideOutputTransformation = new SideOutputTransformation<>(this.getTransformation(), tag); + return new DataStream(this.getExecutionEnvironment(), sideOutputTransformation); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 0dd1b378b6a9e..75972f30d4510 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -26,6 +26,7 @@ import java.util.Map; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; @@ -63,6 +64,7 @@ public class StreamConfig implements Serializable { private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1"; private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2"; private static final String TYPE_SERIALIZER_OUT_1 = "typeSerializer_out"; + private static final String TYPE_SERIALIZER_OUT_PREFIX = "typeSerializer_"; private static final String ITERATON_WAIT = "iterationWait"; private static final String NONCHAINED_OUTPUTS = "nonChainedOutputs"; private static final String EDGES_IN_ORDER = "edgesInOrder"; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java index e2bcac18956c0..c58e4477ce82b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java @@ -21,6 +21,8 @@ import java.util.List; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; /** @@ -43,6 +45,8 @@ public class StreamEdge implements Serializable { */ final private int typeNumber; + private final TypeInformation outputType; + /** * A list of output names that the target vertex listens to (if there is * output selection). @@ -51,12 +55,13 @@ public class StreamEdge implements Serializable { private StreamPartitioner outputPartitioner; public StreamEdge(StreamNode sourceVertex, StreamNode targetVertex, int typeNumber, - List selectedNames, StreamPartitioner outputPartitioner) { + List selectedNames, StreamPartitioner outputPartitioner, TypeInformation outputType) { this.sourceVertex = sourceVertex; this.targetVertex = targetVertex; this.typeNumber = typeNumber; this.selectedNames = selectedNames; this.outputPartitioner = outputPartitioner; + this.outputType = outputType; this.edgeId = sourceVertex + "_" + targetVertex + "_" + typeNumber + "_" + selectedNames + "_" + outputPartitioner; @@ -86,6 +91,8 @@ public List getSelectedNames() { return selectedNames; } + public TypeInformation getOutputType() { return outputType; } + public StreamPartitioner getPartitioner() { return outputPartitioner; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index c946e987fff2d..ab9a758043e70 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.OutputTagUtil; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; @@ -392,7 +393,7 @@ private void addEdgeInternal(Integer upStreamVertexID, } } - StreamEdge edge = new StreamEdge(upstreamNode, downstreamNode, typeNumber, outputNames, partitioner); + StreamEdge edge = new StreamEdge(upstreamNode, downstreamNode, typeNumber, outputNames, partitioner, OutputTagUtil.getSideOutputTypeInfo(outputNames)); getStreamNode(edge.getSourceId()).addOutEdge(edge); getStreamNode(edge.getTargetId()).addInEdge(edge); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java index 506b664622b36..7dc395254c241 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java @@ -18,22 +18,13 @@ package org.apache.flink.streaming.api.graph; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.OutputTagUtil; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; -import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation; -import org.apache.flink.streaming.api.transformations.FeedbackTransformation; -import org.apache.flink.streaming.api.transformations.OneInputTransformation; -import org.apache.flink.streaming.api.transformations.PartitionTransformation; -import org.apache.flink.streaming.api.transformations.SelectTransformation; -import org.apache.flink.streaming.api.transformations.SinkTransformation; -import org.apache.flink.streaming.api.transformations.SourceTransformation; -import org.apache.flink.streaming.api.transformations.SplitTransformation; -import org.apache.flink.streaming.api.transformations.StreamTransformation; -import org.apache.flink.streaming.api.transformations.TwoInputTransformation; -import org.apache.flink.streaming.api.transformations.UnionTransformation; +import org.apache.flink.streaming.api.transformations.*; import org.apache.flink.util.MathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -194,7 +185,10 @@ private Collection transform(StreamTransformation transform) { transformedIds = transformCoFeedback((CoFeedbackTransformation) transform); } else if (transform instanceof PartitionTransformation) { transformedIds = transformPartition((PartitionTransformation) transform); - } else { + } else if (transform instanceof SideOutputTransformation) { + transformedIds = transformSideOutput((SideOutputTransformation) transform); + } + else { throw new IllegalStateException("Unknown transformation: " + transform); } @@ -304,6 +298,34 @@ private Collection transformSelect(SelectTransformation select) return virtualResultIds; } + /** + * Transforms a {@code SelectTransformation}. + * + *

+ * For this we create a virtual node in the {@code StreamGraph} holds the selected names. + * @see org.apache.flink.streaming.api.graph.StreamGraphGenerator + */ + private Collection transformSideOutput(SideOutputTransformation sideOutput) { + StreamTransformation input = sideOutput.getInput(); + Collection resultIds = transform(input); + + + // the recursive transform might have already transformed this + if (alreadyTransformed.containsKey(sideOutput)) { + return alreadyTransformed.get(sideOutput); + } + + List virtualResultIds = new ArrayList<>(); + + for (int inputId : resultIds) { + int virtualId = StreamTransformation.getNewNodeId(); + streamGraph.addVirtualSelectNode(inputId, virtualId, OutputTagUtil.getOutputTagName(sideOutput.getOutputTag())); + virtualResultIds.add(virtualId); + } + return virtualResultIds; + } + + /** * Transforms a {@code FeedbackTransformation}. * diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java index 905189161d129..c2e95d862115a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java @@ -19,16 +19,20 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.util.Preconditions; @@ -61,6 +65,7 @@ public class StreamNode implements Serializable { private TypeSerializer typeSerializerIn1; private TypeSerializer typeSerializerIn2; private TypeSerializer typeSerializerOut; + private Map, TypeSerializer> typeSerializerMap; private List inEdges = new ArrayList(); private List outEdges = new ArrayList(); @@ -85,6 +90,7 @@ public StreamNode(StreamExecutionEnvironment env, this.outputSelectors = outputSelector; this.jobVertexClass = jobVertexClass; this.slotSharingGroup = slotSharingGroup; + this.typeSerializerMap = new HashMap<>(); } public void addInEdge(StreamEdge inEdge) { @@ -214,10 +220,18 @@ public TypeSerializer getTypeSerializerOut() { return typeSerializerOut; } + public TypeSerializer getTypeSerializerOut(OutputTag tag) { + return typeSerializerMap.get(tag); + } + public void setSerializerOut(TypeSerializer typeSerializerOut) { this.typeSerializerOut = typeSerializerOut; } + public void setTypeSerializerOut(OutputTag tag, TypeSerializer typeSerializerOut){ + typeSerializerMap.put(tag, typeSerializerOut); + } + public Class getJobVertexClass() { return jobVertexClass; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java index 56fa14d89c0d5..1ab48ef331411 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java @@ -19,8 +19,10 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; +import org.apache.flink.util.RichCollector; /** * Wrapper around an {@link Output} for user functions that expect a {@link Collector}. @@ -32,11 +34,11 @@ * @param The type of the elements that can be emitted. */ @Internal -public class TimestampedCollector implements Collector { +public class TimestampedCollector implements RichCollector { private final Output> output; - private final StreamRecord reuse; + private final StreamRecord reuse; /** * Creates a new {@link TimestampedCollector} that wraps the given {@link Output}. @@ -50,7 +52,12 @@ public TimestampedCollector(Output> output) { public void collect(T record) { output.collect(reuse.replace(record)); } - + + @Override + public void collect(OutputTag tag, S record) { + output.collect(reuse.replace(tag, record)); + } + public void setTimestamp(StreamRecord timestampBase) { if (timestampBase.hasTimestamp()) { reuse.setTimestamp(timestampBase.getTimestamp()); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java new file mode 100644 index 0000000000000..057b46fda32e2 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java @@ -0,0 +1,62 @@ +/** + * 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.transformations; + +import com.google.common.collect.Lists; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; +import org.apache.flink.streaming.api.operators.ChainingStrategy; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +public class SideOutputTransformation extends StreamTransformation { + private final StreamTransformation input; + private final OutputTag tag; + + public SideOutputTransformation(StreamTransformation input, final OutputTag tag) { + super("SideOutput", tag.getTypeInfo(), input.getParallelism()); + this.input = input; + this.tag = tag; + } + + /** + * Returns the input {@code StreamTransformation}. + */ + public StreamTransformation getInput() { + return input; + } + + public OutputTag getOutputTag() { + return tag; + } + + @Override + public Collection> getTransitivePredecessors() { + List> result = Lists.newArrayList(); + result.add(this); + result.addAll(input.getTransitivePredecessors()); + return result; + } + + @Override + public final void setChainingStrategy(ChainingStrategy strategy) { + throw new UnsupportedOperationException("Cannot set chaining strategy on Select Transformation."); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index 9f046f659a1bd..b9ff18d9f7131 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -44,6 +44,7 @@ public class RecordWriterOutput implements Output> { private SerializationDelegate serializationDelegate; + private TypeSerializer outRecordSerializer; @SuppressWarnings("unchecked") public RecordWriterOutput( @@ -52,13 +53,12 @@ public RecordWriterOutput( boolean enableMultiplexing) { checkNotNull(recordWriter); - + // generic hack: cast the writer to generic Object type so we can use it // with multiplexed records and watermarks this.recordWriter = (StreamRecordWriter>) (StreamRecordWriter) recordWriter; - TypeSerializer outRecordSerializer; if (enableMultiplexing) { outRecordSerializer = new MultiplexingStreamRecordSerializer(outSerializer); } else { @@ -73,14 +73,15 @@ public RecordWriterOutput( @Override public void collect(StreamRecord record) { - serializationDelegate.setInstance(record); try { + serializationDelegate.setInstance(record); recordWriter.emit(serializationDelegate); + } catch (Exception e) { + //Hack, should do type check + //throw new RuntimeException(e.getMessage(), e); } - catch (Exception e) { - throw new RuntimeException(e.getMessage(), e); - } + } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java index 9f751616c7c46..ed9d6f2700546 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.runtime.streamrecord; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; +import org.apache.flink.api.common.typeinfo.TypeInformation; /** * One value in a data stream. This stores the value and an optional associated timestamp. @@ -35,6 +37,8 @@ public final class StreamRecord extends StreamElement { /** Flag whether the timestamp is actually set */ private boolean hasTimestamp; + + protected OutputTag tag; /** * Creates a new StreamRecord. The record does not have a timestamp. @@ -43,6 +47,8 @@ public StreamRecord(T value) { this.value = value; } + public TypeInformation getType() { return this.tag == null ? null : this.tag.getTypeInfo();} + /** * Creates a new StreamRecord wrapping the given value. The timestamp is set to the * given timestamp. @@ -107,6 +113,12 @@ public StreamRecord replace(X element) { return (StreamRecord) this; } + @SuppressWarnings("unchecked") + public StreamRecord replace(OutputTag tag, X element) { + this.value = (T) element; + return (StreamRecord) this; + } + /** * Replace the currently stored value by the given new value and the currently stored * timestamp with the new timestamp. This returns a StreamElement with the generic type diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index 7342b6d86988b..cee39f8c2c013 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -25,6 +25,8 @@ import java.util.Random; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; +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.metrics.Counter; @@ -96,7 +98,7 @@ public OperatorChain(StreamTask containingTask, AccumulatorRegistry.Rep try { for (int i = 0; i < outEdgesInOrder.size(); i++) { StreamEdge outEdge = outEdgesInOrder.get(i); - + RecordWriterOutput streamOutput = createStreamOutput( outEdge, chainedConfigs.get(outEdge.getSourceId()), i, containingTask.getEnvironment(), enableMultiplexing, reporter, containingTask.getName()); @@ -304,9 +306,9 @@ private static Output> createChainedOperator( } private static RecordWriterOutput createStreamOutput( - StreamEdge edge, StreamConfig upStreamConfig, int outputIndex, - Environment taskEnvironment, boolean enableMultiplexing, - AccumulatorRegistry.Reporter reporter, String taskName) + StreamEdge edge, StreamConfig upStreamConfig, int outputIndex, + Environment taskEnvironment, boolean enableMultiplexing, + AccumulatorRegistry.Reporter reporter, String taskName) { TypeSerializer outSerializer = upStreamConfig.getTypeSerializerOut(taskEnvironment.getUserClassLoader()); @@ -393,14 +395,18 @@ public CopyingChainingOutput(OneInputStreamOperator operator, TypeSerializ @Override public void collect(StreamRecord record) { - try { - numRecordsIn.inc(); - StreamRecord copy = record.copy(serializer.copy(record.getValue())); - operator.setKeyContextElement1(copy); - operator.processElement(copy); - } - catch (Exception e) { - throw new RuntimeException("Could not forward element to next operator", e); + //Hack, shoud do better type check + if(record.getValue().getClass() == serializer.createInstance().getClass()) { + try { + numRecordsIn.inc(); + StreamRecord copy = record.copy(serializer.copy(record.getValue())); + operator.setKeyContextElement1(copy); + operator.processElement(copy); + } catch (Exception e) { + throw new RuntimeException("Could not forward element to next operator", e); + } + } else { + LOG.info("{} {}",record.getValue().getClass().toString(), serializer.createInstance().getClass().toString()); } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 3dd2ed7ba993b..ac647b5769909 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -455,6 +455,7 @@ private void configureChainedTestingStreamOperator( ), 0, Collections.emptyList(), + null, null ); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index cbb5a9ddf4dca..c45dbdb262299 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -142,7 +142,7 @@ private void initializeOutput() { StreamNode sourceVertexDummy = new StreamNode(null, 0, "group", dummyOperator, "source dummy", new LinkedList>(), SourceStreamTask.class); StreamNode targetVertexDummy = new StreamNode(null, 1, "group", dummyOperator, "target dummy", new LinkedList>(), SourceStreamTask.class); - outEdgesInOrder.add(new StreamEdge(sourceVertexDummy, targetVertexDummy, 0, new LinkedList(), new BroadcastPartitioner())); + outEdgesInOrder.add(new StreamEdge(sourceVertexDummy, targetVertexDummy, 0, new LinkedList(), new BroadcastPartitioner(), null)); streamConfig.setOutEdgesInOrder(outEdgesInOrder); streamConfig.setNonChainedOutputs(outEdgesInOrder); streamConfig.setTypeSerializerOut(outputSerializer); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java index 0e7565ef56a9d..9609dc64e872a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java @@ -133,7 +133,8 @@ protected void initializeInputs() throws IOException, InterruptedException { targetVertexDummy, 1, new LinkedList(), - new BroadcastPartitioner()); + new BroadcastPartitioner(), + null); inPhysicalEdges.add(streamEdge); break; @@ -148,7 +149,8 @@ protected void initializeInputs() throws IOException, InterruptedException { targetVertexDummy, 2, new LinkedList(), - new BroadcastPartitioner()); + new BroadcastPartitioner(), + null); inPhysicalEdges.add(streamEdge); break; From 3ca09be6401e3c56256a54ad4d930734320db624 Mon Sep 17 00:00:00 2001 From: Chen Qin Date: Thu, 1 Dec 2016 14:19:56 -0800 Subject: [PATCH 2/5] allow mutiple OutputTag with same type implement windowopeator late arriving events add unit/integration tests --- .../flink/api/common/typeinfo/OutputTag.java | 52 ++++- .../flink/api/common/typeinfo/TypeHint.java | 2 +- .../api/common/typeutils/OutputTagUtil.java | 27 --- .../apache/flink/util/CollectorWrapper.java | 77 +++++++ .../org/apache/flink/util/RichCollector.java | 12 +- .../examples/wordcount/SideOutputExample.java | 126 +++++++++++ .../examples/wordcount/WordCount.java | 24 +-- .../api/datastream/AllWindowedStream.java | 10 + .../streaming/api/datastream/DataStream.java | 120 +++++------ .../SingleOutputStreamOperator.java | 26 ++- .../api/datastream/WindowedStream.java | 10 + .../windowing/DiscardAllWindowFunction.java | 32 +++ .../windowing/DiscardWindowFunction.java | 32 +++ .../streaming/api/graph/StreamConfig.java | 27 ++- .../flink/streaming/api/graph/StreamEdge.java | 15 +- .../streaming/api/graph/StreamGraph.java | 46 +++- .../api/graph/StreamGraphGenerator.java | 7 +- .../flink/streaming/api/graph/StreamNode.java | 12 +- .../api/graph/StreamingJobGraphGenerator.java | 12 ++ .../SideOutputTransformation.java | 15 +- .../runtime/io/RecordWriterOutput.java | 18 +- .../operators/windowing/WindowOperator.java | 28 +++ .../runtime/streamrecord/StreamRecord.java | 54 ++++- .../runtime/tasks/OperatorChain.java | 33 +-- .../outputtags/LateArrivingOutputTag.java | 34 +++ .../api/operators/StreamFlatMapTest.java | 9 + .../api/operators/co/CoStreamFlatMapTest.java | 8 + .../windowing/WindowOperatorTest.java | 91 ++++++-- .../streamrecord/StreamRecordTest.java | 28 +++ .../AbstractStreamOperatorTestHarness.java | 41 +++- .../api/scala/AllWindowedStream.scala | 17 +- .../streaming/api/scala/DataStream.scala | 7 +- .../streaming/api/scala/WindowedStream.scala | 15 +- .../streaming/runtime/SideOutputITCase.java | 204 ++++++++++++++++++ 34 files changed, 1054 insertions(+), 217 deletions(-) delete mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java create mode 100644 flink-core/src/main/java/org/apache/flink/util/CollectorWrapper.java create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/SideOutputExample.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java index 6e5ea0195f74b..6ee4d15be3b73 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java @@ -17,19 +17,63 @@ */ package org.apache.flink.api.common.typeinfo; +import static java.util.Objects.requireNonNull; import java.io.Serializable; +import org.apache.flink.annotation.PublicEvolving; + + /** - * Created by chenqin on 10/20/16. + * Sideoutput meta info, used to split sideoutput next to operator */ +@PublicEvolving public abstract class OutputTag extends TypeHint implements Serializable{ + private static final long serialVersionUID = 1L; + + private final T value; + + /** + * reserved constructor for {@code LateArrivingOutputTag} + */ + protected OutputTag() { + this.value = null; + } + + /** + * assign value of typed outputtag + * allow expose more than one outputtag with same type + * @param value outputtag value + */ + public OutputTag(T value) { + this.value = requireNonNull(value); + } + + /** + * get value of outputtag + * @return outputtag value + */ + public T getValue() { + return value; + } + + @Override public boolean equals(Object obj) { - return super.equals(obj); + return obj instanceof OutputTag + && ((OutputTag)obj).getTypeInfo().equals(this.getTypeInfo()) + && (((OutputTag)obj).value != null && ((OutputTag)obj).value.equals(this.value) + || (((OutputTag)obj).value == null && this.value == null)); } - public void setTypeInfo(TypeInformation t) { - this.typeInfo = t; + @Override + public int hashCode() { + int result = value != null ? value.hashCode() : 0; + return 31 * result + (getTypeInfo().hashCode() ^ (getTypeInfo().hashCode() >>> 32)); + } + + @Override + public String toString() { + return getTypeInfo().toString() + "@" + (value == null ? "undefined" : value.toString()); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java index b84935a62c147..975d6e36942f2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java @@ -38,7 +38,7 @@ public abstract class TypeHint { /** The type information described by the hint */ - protected TypeInformation typeInfo; + private final TypeInformation typeInfo; /** * Creates a hint for the generic type in the class signature. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java deleted file mode 100644 index cca468917fda7..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/OutputTagUtil.java +++ /dev/null @@ -1,27 +0,0 @@ -package org.apache.flink.api.common.typeutils; - -import org.apache.flink.api.common.typeinfo.OutputTag; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.TypeInfoParser; - -import java.util.Arrays; -import java.util.List; - - -/** - * Created by chenqin on 10/21/16. - */ -public class OutputTagUtil { - public static List getOutputTagName(OutputTag tag){ - return Arrays.asList(tag.getTypeInfo().toString()); - } - - public static TypeInformation getSideOutputTypeInfo(List names){ - //HACK Alert - if(names.size() == 1 && names.get(0) == "String") { - return TypeInfoParser.parse(names.get(0)); - } else { - return null; - } - } -} diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectorWrapper.java b/flink-core/src/main/java/org/apache/flink/util/CollectorWrapper.java new file mode 100644 index 0000000000000..5f050fc2ba9f3 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/CollectorWrapper.java @@ -0,0 +1,77 @@ +/* + * 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.util; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.OutputTag; + +import java.io.Serializable; + +/** + * Wrapper to collect normal output and sideoutput. Provide Wrapper to sideoutput element + * without break 1.x API compatability + * + * Example: + * flatMap(T value, Collector<OUT> collector) { + * CollectorWrapper<OUT> wrapper = CollectorWrapper<>(collector); + * wrapper.collect(...)// same as collector.collect(..) + * wrapper.collect(OutputTag<SIDEOUT> sideOutputTag, ...) //sideOutput + * } + * + */ +@PublicEvolving +public class CollectorWrapper implements RichCollector , Serializable{ + + private final RichCollector collector; + + /** + * side output wrapper, allow user do sideoutputs + * @param collector public collector interface + */ + public CollectorWrapper(final Collector collector) { + this.collector = (RichCollector) collector; + } + + /** + * side output collect interface + * @param outputTag side output outputTag + * @param value side output element + * @param sideoutput type + */ + @Override + public void collect(final OutputTag outputTag, final S value) { + collector.collect(outputTag, value); + } + + /** + * backward compatabile, output non sideoutputs + * @param record The record to collect. + */ + @Override + public void collect(T record) { + this.collector.collect(record); + } + + /** + * close collector + */ + @Override + public void close() { + this.collector.close(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/util/RichCollector.java b/flink-core/src/main/java/org/apache/flink/util/RichCollector.java index 476a8b7d3bfa5..6e17073c719b3 100644 --- a/flink-core/src/main/java/org/apache/flink/util/RichCollector.java +++ b/flink-core/src/main/java/org/apache/flink/util/RichCollector.java @@ -18,13 +18,19 @@ package org.apache.flink.util; -import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.OutputTag; /** - * support collect with outputtag + * extend support collect with outputtag */ -@Public +@Internal public interface RichCollector extends Collector{ + /** + * collect side output element with a specific outputtag + * @param tag side output outputtag + * @param value side output element + * @param sideoutput class type information + */ void collect(OutputTag tag, S value); } diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/SideOutputExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/SideOutputExample.java new file mode 100644 index 0000000000000..02d30087e8eae --- /dev/null +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/SideOutputExample.java @@ -0,0 +1,126 @@ +/* + * 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.examples.wordcount; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.OutputTag; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.examples.java.wordcount.util.WordCountData; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Collector; +import org.apache.flink.util.CollectorWrapper; + +/** + * Implements the "WordCount" program that computes a simple word occurrence + * histogram over text files in a streaming fashion. At same time, write input + * to sideOutput channel and write to another stream + *

+ * The input is a plain text file with lines separated by newline characters. + * + *

+ * Usage: WordCount --input <path> --output <path>
+ * If no parameters are provided, the program is run with default data from + * {@link WordCountData}. + * + *

+ * This example shows how to: + *

    + *
  • write a simple Flink Streaming program, + *
  • use tuple data types, + *
  • write and use user-defined functions. + *
+ * + */ +public class SideOutputExample { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + // Checking input parameters + final ParameterTool params = ParameterTool.fromArgs(args); + + // set up the execution environment + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // make parameters available in the web interface + env.getConfig().setGlobalJobParameters(params); + + // get input data + DataStream text; + if (params.has("input")) { + // read the text file from given input path + text = env.readTextFile(params.get("input")); + } else { + System.out.println("Executing WordCount example with default input data set."); + System.out.println("Use --input to specify file input."); + // get default test text data + text = env.fromElements(WordCountData.WORDS); + } + + DataStream sideOutput = text.flatMap(new Tokenizer()).getSideOutput(sideOutTag); + + // emit result + if (params.has("output")) { + sideOutput.writeAsText(params.get("sideoutput")); + } else { + System.out.println("Printing result to stdout. Use --output to specify output path."); + sideOutput.print(); + } + + // execute program + env.execute("Streaming WordCount SideOutput"); + } + + static final OutputTag sideOutTag = new OutputTag("sideout") {}; + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** + * Implements the string tokenizer that splits sentences into words as a + * user-defined FlatMapFunction. The function takes a line (String) and + * splits it into multiple pairs in the form of "(word,1)" ({@code Tuple2}). + */ + public static final class Tokenizer implements FlatMapFunction> { + private static final long serialVersionUID = 1L; + + @Override + public void flatMap(String value, Collector> out) + throws Exception { + CollectorWrapper wrapper = new CollectorWrapper<>(out); + // normalize and split the line + String[] tokens = value.toLowerCase().split("\\W+"); + + // emit the pairs + for (String token : tokens) { + if (token.length() > 0) { + wrapper.collect(new Tuple2(token, 1)); + } + wrapper.collect(sideOutTag, token); + } + } + } + +} diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java index 160eede589465..5ec1b22ec47f4 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java @@ -18,27 +18,25 @@ package org.apache.flink.streaming.examples.wordcount; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; -import org.apache.flink.util.RichCollector; /** * Implements the "WordCount" program that computes a simple word occurrence * histogram over text files in a streaming fashion. - * + * *

* The input is a plain text file with lines separated by newline characters. - * + * *

* Usage: WordCount --input <path> --output <path>
* If no parameters are provided, the program is run with default data from * {@link WordCountData}. - * + * *

* This example shows how to: *

    @@ -46,7 +44,7 @@ *
  • use tuple data types, *
  • write and use user-defined functions. *
- * + * */ public class WordCount { @@ -78,13 +76,11 @@ public static void main(String[] args) throws Exception { } DataStream> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) - text.flatMap(new Tokenizer()) - // group by the tuple field "0" and sum up tuple field "1" + // split up the lines in pairs (2-tuples) containing: (word,1) + text.flatMap(new Tokenizer()) + // group by the tuple field "0" and sum up tuple field "1" .keyBy(0).sum(1); - text.flatMap(new Tokenizer()).getOutput(sideOut).print(); - // emit result if (params.has("output")) { counts.writeAsText(params.get("output")); @@ -97,8 +93,6 @@ public static void main(String[] args) throws Exception { env.execute("Streaming WordCount"); } - static final OutputTag sideOut = new OutputTag() {}; - // ************************************************************************* // USER FUNCTIONS // ************************************************************************* @@ -114,7 +108,7 @@ public static final class Tokenizer implements FlatMapFunction> out) - throws Exception { + throws Exception { // normalize and split the line String[] tokens = value.toLowerCase().split("\\W+"); @@ -124,8 +118,6 @@ public void flatMap(String value, Collector> out) out.collect(new Tuple2(token, 1)); } } - RichCollector> sideout = (RichCollector>) out; - sideout.collect(sideOut, "sideout"); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java index ae71ce53a5f67..5ad60ee077f90 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java @@ -36,6 +36,7 @@ import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator; import org.apache.flink.streaming.api.functions.aggregation.SumAggregator; import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.DiscardAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.PassThroughAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.FoldApplyAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ReduceApplyAllWindowFunction; @@ -52,6 +53,7 @@ import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueAllWindowFunction; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; /** * A {@code AllWindowedStream} represents a data stream where the stream of @@ -440,6 +442,14 @@ public SingleOutputStreamOperator apply(AllWindowFunction functi return apply(function, resultType); } + /** + * Applies a DiscardWindowFunction that only returns late arriving events + * @return the data stream considered too late to be evaluated by any windows assigned + */ + public DataStream> tooLateEvents() { + return apply(new DiscardAllWindowFunction()).getSideOutput(new LateArrivingOutputTag()); + } + /** * Applies the given window function to each window. The window function is called for each * evaluation of the window for each key individually. The output of the window function is diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 703f6ad1b89f7..7f25bb249ecbb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -17,9 +17,12 @@ package org.apache.flink.streaming.api.datastream; +import java.util.ArrayList; +import java.util.List; + +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; @@ -29,7 +32,6 @@ import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -37,6 +39,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.io.CsvOutputFormat; import org.apache.flink.api.java.io.TextOutputFormat; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -47,13 +50,13 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; +import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; import org.apache.flink.streaming.api.functions.TimestampExtractor; import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.sink.SocketClientSink; -import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; -import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.streaming.api.operators.StreamFlatMap; @@ -64,10 +67,10 @@ import org.apache.flink.streaming.api.transformations.StreamTransformation; import org.apache.flink.streaming.api.transformations.UnionTransformation; import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; -import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows; import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows; -import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows; import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; import org.apache.flink.streaming.api.windowing.evictors.CountEvictor; import org.apache.flink.streaming.api.windowing.time.Time; @@ -82,18 +85,15 @@ import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; -import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; -import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.apache.flink.streaming.runtime.partitioner.RescalePartitioner; +import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; +import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.util.keys.KeySelectorUtil; import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.apache.flink.util.Preconditions; -import java.util.ArrayList; -import java.util.List; - /** * A DataStream represents a stream of elements of the same type. A DataStream * can be transformed into another DataStream by applying a transformation as @@ -192,9 +192,9 @@ public final DataStream union(DataStream... streams) { for (DataStream newStream : streams) { if (!getType().equals(newStream.getType())) { throw new IllegalArgumentException("Cannot union streams of different types: " - + getType() + " and " + newStream.getType()); + + getType() + " and " + newStream.getType()); } - + unionedTransforms.add(newStream.getTransformation()); } return new DataStream<>(this.environment, new UnionTransformation<>(unionedTransforms)); @@ -232,7 +232,7 @@ public ConnectedStreams connect(DataStream dataStream) { /** * * It creates a new {@link KeyedStream} that uses the provided key for partitioning - * its operator states. + * its operator states. * * @param key * The KeySelector to be used for extracting the key for partitioning @@ -243,7 +243,7 @@ public KeyedStream keyBy(KeySelector key) { } /** - * Partitions the operator state of a {@link DataStream} by the given key positions. + * Partitions the operator state of a {@link DataStream} by the given key positions. * * @param fields * The position of the fields on which the {@link DataStream} @@ -259,7 +259,7 @@ public KeyedStream keyBy(int... fields) { } /** - * Partitions the operator state of a {@link DataStream}using field expressions. + * Partitions the operator state of a {@link DataStream}using field expressions. * A field expression is either the name of a public field or a getter method with parentheses * of the {@link DataStream}S underlying type. A dot can be used to drill * down into objects, as in {@code "field1.getInnerField2()" }. @@ -275,7 +275,7 @@ public KeyedStream keyBy(String... fields) { private KeyedStream keyBy(Keys keys) { return new KeyedStream<>(this, clean(KeySelectorUtil.getSelectorForKeys(keys, - getType(), getExecutionConfig()))); + getType(), getExecutionConfig()))); } /** @@ -326,7 +326,7 @@ public DataStream partitionCustom(Partitioner partitioner, String fiel */ public DataStream partitionCustom(Partitioner partitioner, KeySelector keySelector) { return setConnectionType(new CustomPartitionerWrapper<>(clean(partitioner), - clean(keySelector))); + clean(keySelector))); } // private helper method for custom partitioning @@ -334,9 +334,9 @@ private DataStream partitionCustom(Partitioner partitioner, Keys ke KeySelector keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig()); return setConnectionType( - new CustomPartitionerWrapper<>( - clean(partitioner), - clean(keySelector))); + new CustomPartitionerWrapper<>( + clean(partitioner), + clean(keySelector))); } /** @@ -504,7 +504,7 @@ public IterativeStream iterate(long maxWaitTimeMillis) { public SingleOutputStreamOperator map(MapFunction mapper) { TypeInformation outType = TypeExtractor.getMapReturnTypes(clean(mapper), getType(), - Utils.getCallLocationName(), true); + Utils.getCallLocationName(), true); return transform("Map", outType, new StreamMap<>(clean(mapper))); } @@ -528,7 +528,7 @@ public SingleOutputStreamOperator map(MapFunction mapper) { public SingleOutputStreamOperator flatMap(FlatMapFunction flatMapper) { TypeInformation outType = TypeExtractor.getFlatMapReturnTypes(clean(flatMapper), - getType(), Utils.getCallLocationName(), true); + getType(), Utils.getCallLocationName(), true); return transform("Flat Map", outType, new StreamFlatMap<>(clean(flatMapper))); @@ -667,8 +667,8 @@ public AllWindowedStream countWindowAll(long size) { */ public AllWindowedStream countWindowAll(long size, long slide) { return windowAll(GlobalWindows.create()) - .evictor(CountEvictor.of(size)) - .trigger(CountTrigger.of(slide)); + .evictor(CountEvictor.of(size)) + .trigger(CountTrigger.of(slide)); } /** @@ -698,7 +698,7 @@ public AllWindowedStream windowAll(WindowAssigner AllWindowedStream windowAll(WindowAssigner assignTimestamps(TimestampExtractor extr int inputParallelism = getTransformation().getParallelism(); ExtractTimestampsOperator operator = new ExtractTimestampsOperator<>(clean(extractor)); return transform("ExtractTimestamps", getTransformation().getOutputType(), operator) - .setParallelism(inputParallelism); + .setParallelism(inputParallelism); } /** * Assigns timestamps to the elements in the data stream and periodically creates * watermarks to signal event time progress. - * + * *

This method creates watermarks periodically (for example every second), based * on the watermarks indicated by the given watermark generator. Even when no new elements * in the stream arrive, the given watermark generator will be periodically checked for * new watermarks. The interval in which watermarks are generated is defined in * {@link ExecutionConfig#setAutoWatermarkInterval(long)}. - * + * *

Use this method for the common cases, where some characteristic over all elements * should generate the watermarks, or where watermarks are simply trailing behind the * wall clock time by a certain amount. @@ -746,33 +746,33 @@ public SingleOutputStreamOperator assignTimestamps(TimestampExtractor extr * timestamp seen so far in the elements of the stream by a fixed amount of time, and this * amount is known in advance, use the * {@link BoundedOutOfOrdernessTimestampExtractor}. - * + * *

For cases where watermarks should be created in an irregular fashion, for example * based on certain markers that some element carry, use the * {@link AssignerWithPunctuatedWatermarks}. - * + * * @param timestampAndWatermarkAssigner The implementation of the timestamp assigner and - * watermark generator. + * watermark generator. * @return The stream after the transformation, with assigned timestamps and watermarks. - * + * * @see AssignerWithPeriodicWatermarks * @see AssignerWithPunctuatedWatermarks - * @see #assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks) + * @see #assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks) */ public SingleOutputStreamOperator assignTimestampsAndWatermarks( - AssignerWithPeriodicWatermarks timestampAndWatermarkAssigner) { - + AssignerWithPeriodicWatermarks timestampAndWatermarkAssigner) { + // match parallelism to input, otherwise dop=1 sources could lead to some strange // behaviour: the watermark will creep along very slowly because the elements // from the source go to each extraction operator round robin. final int inputParallelism = getTransformation().getParallelism(); final AssignerWithPeriodicWatermarks cleanedAssigner = clean(timestampAndWatermarkAssigner); - - TimestampsAndPeriodicWatermarksOperator operator = - new TimestampsAndPeriodicWatermarksOperator<>(cleanedAssigner); - + + TimestampsAndPeriodicWatermarksOperator operator = + new TimestampsAndPeriodicWatermarksOperator<>(cleanedAssigner); + return transform("Timestamps/Watermarks", getTransformation().getOutputType(), operator) - .setParallelism(inputParallelism); + .setParallelism(inputParallelism); } /** @@ -786,7 +786,7 @@ public SingleOutputStreamOperator assignTimestampsAndWatermarks( * non-negative and greater than the previous watermark. * *

This method is useful when the data stream embeds watermark elements, or certain elements - * carry a marker that can be used to determine the current event time watermark. + * carry a marker that can be used to determine the current event time watermark. * This operation gives the programmer full control over the watermark generation. Users * should be aware that too aggressive watermark generation (i.e., generating hundreds of * watermarks every second) can cost some performance. @@ -795,7 +795,7 @@ public SingleOutputStreamOperator assignTimestampsAndWatermarks( * every x milliseconds, use the {@link AssignerWithPeriodicWatermarks}. * * @param timestampAndWatermarkAssigner The implementation of the timestamp assigner and - * watermark generator. + * watermark generator. * @return The stream after the transformation, with assigned timestamps and watermarks. * * @see AssignerWithPunctuatedWatermarks @@ -803,25 +803,25 @@ public SingleOutputStreamOperator assignTimestampsAndWatermarks( * @see #assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks) */ public SingleOutputStreamOperator assignTimestampsAndWatermarks( - AssignerWithPunctuatedWatermarks timestampAndWatermarkAssigner) { - + AssignerWithPunctuatedWatermarks timestampAndWatermarkAssigner) { + // match parallelism to input, otherwise dop=1 sources could lead to some strange // behaviour: the watermark will creep along very slowly because the elements // from the source go to each extraction operator round robin. final int inputParallelism = getTransformation().getParallelism(); final AssignerWithPunctuatedWatermarks cleanedAssigner = clean(timestampAndWatermarkAssigner); - TimestampsAndPunctuatedWatermarksOperator operator = - new TimestampsAndPunctuatedWatermarksOperator<>(cleanedAssigner); - + TimestampsAndPunctuatedWatermarksOperator operator = + new TimestampsAndPunctuatedWatermarksOperator<>(cleanedAssigner); + return transform("Timestamps/Watermarks", getTransformation().getOutputType(), operator) - .setParallelism(inputParallelism); + .setParallelism(inputParallelism); } // ------------------------------------------------------------------------ // Data sinks // ------------------------------------------------------------------------ - + /** * Writes a DataStream to the standard output stream (stdout). * @@ -954,10 +954,10 @@ public DataStreamSink writeAsCsv(String path, WriteMode writeMode) { @SuppressWarnings("unchecked") @PublicEvolving public DataStreamSink writeAsCsv( - String path, - WriteMode writeMode, - String rowDelimiter, - String fieldDelimiter) { + String path, + WriteMode writeMode, + String rowDelimiter, + String fieldDelimiter) { Preconditions.checkArgument( getType().isTupleType(), "The writeAsCsv() method can only be used on data streams of tuples."); @@ -1029,11 +1029,11 @@ public SingleOutputStreamOperator transform(String operatorName, TypeInfo transformation.getOutputType(); OneInputTransformation resultTransform = new OneInputTransformation<>( - this.transformation, - operatorName, - operator, - outTypeInfo, - environment.getParallelism()); + this.transformation, + operatorName, + operator, + outTypeInfo, + environment.getParallelism()); @SuppressWarnings({ "unchecked", "rawtypes" }) SingleOutputStreamOperator returnStream = new SingleOutputStreamOperator(environment, resultTransform); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java index 5bf92e5170f66..e7b980728e2be 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java @@ -344,8 +344,28 @@ public SingleOutputStreamOperator slotSharingGroup(String slotSharingGroup) { return this; } - public DataStream getOutput(OutputTag tag){ - SideOutputTransformation sideOutputTransformation = new SideOutputTransformation<>(this.getTransformation(), tag); - return new DataStream(this.getExecutionEnvironment(), sideOutputTransformation); + /** + * Gets DataStream of elements with sideOutputTag attached. + * @param sideOutputTag OutputTag used to sideOutput elements + * @param type of sideOutputTag same as elements type + * @return DataStream of sideoutput elements + * Example: + * + * static final OutputTag<X> sideOutputTag = new OutputTag<X>() {}; + * public void flatMap(X value, Collector<String> out) throws Exception { + * CollectorWrapper wrapper = new CollectorWrapper<>(out); + * try{ + * ... + * wrapper.collect(value.toString()); + * } catch(Exception e) { + * wrapper.collect(sideOutputTag, value); + * } + * } + * DataStream<X> sideOutputStream = ...flatMap(...).getSideOutput(sideOut); + * + */ + public DataStream getSideOutput(OutputTag sideOutputTag){ + SideOutputTransformation sideOutputTransformation = new SideOutputTransformation<>(this.getTransformation(), requireNonNull(sideOutputTag)); + return new DataStream<>(this.getExecutionEnvironment(), sideOutputTransformation); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java index ad7f371b87c83..f0c3ca58703ed 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java @@ -36,6 +36,7 @@ import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction; import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator; import org.apache.flink.streaming.api.functions.aggregation.SumAggregator; +import org.apache.flink.streaming.api.functions.windowing.DiscardWindowFunction; import org.apache.flink.streaming.api.functions.windowing.FoldApplyWindowFunction; import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ReduceApplyWindowFunction; @@ -59,6 +60,7 @@ import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; /** * A {@code WindowedStream} represents a data stream where elements are grouped by @@ -723,6 +725,14 @@ public SingleOutputStreamOperator apply(R initialValue, FoldFunction> tooLateEvents() { + return apply(new DiscardWindowFunction()).getSideOutput(new LateArrivingOutputTag()); + } + // ------------------------------------------------------------------------ // Aggregations on the keyed windows // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java new file mode 100644 index 0000000000000..aafc0e7291b8e --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java @@ -0,0 +1,32 @@ +/** + * 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.windowing; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.windowing.windows.Window; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.Collector; + +/** + * discard non sideoutputs + */ +@Internal +public class DiscardAllWindowFunction implements AllWindowFunction{ + @Override + public void apply(W window, Iterable values, Collector out) throws Exception {} +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java new file mode 100644 index 0000000000000..5ce3b3d547f13 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java @@ -0,0 +1,32 @@ +/** + * 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.windowing; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.windowing.windows.Window; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.Collector; + +/** + * discard non sideoutputs of keyed window, pass through sideOutputs only + */ +@Internal +public class DiscardWindowFunction implements WindowFunction{ + @Override + public void apply(K k, W window, Iterable input, Collector out) throws Exception {} +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 2c0255c3a590f..4ed683b5e0799 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -17,6 +17,14 @@ package org.apache.flink.streaming.api.graph; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; @@ -31,14 +39,6 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskException; import org.apache.flink.util.InstantiationUtil; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - @Internal public class StreamConfig implements Serializable { @@ -63,7 +63,7 @@ public class StreamConfig implements Serializable { private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1"; private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2"; private static final String TYPE_SERIALIZER_OUT_1 = "typeSerializer_out"; - private static final String TYPE_SERIALIZER_OUT_PREFIX = "typeSerializer_"; + private static final String TYPE_SERIALIZER_SIDEOUT_PREFIX = "typeSerializer_sideout_"; private static final String ITERATON_WAIT = "iterationWait"; private static final String NONCHAINED_OUTPUTS = "nonChainedOutputs"; private static final String EDGES_IN_ORDER = "edgesInOrder"; @@ -140,6 +140,10 @@ public void setTypeSerializerIn2(TypeSerializer serializer) { public void setTypeSerializerOut(TypeSerializer serializer) { setTypeSerializer(TYPE_SERIALIZER_OUT_1, serializer); } + + public void setTypeSerializerSideOuts(String typeString, TypeSerializer serializer) { + setTypeSerializer(TYPE_SERIALIZER_SIDEOUT_PREFIX + typeString, serializer); + } public TypeSerializer getTypeSerializerIn1(ClassLoader cl) { try { @@ -157,9 +161,10 @@ public TypeSerializer getTypeSerializerIn2(ClassLoader cl) { } } - public TypeSerializer getTypeSerializerOut(ClassLoader cl) { + public TypeSerializer getTypeSerializerOut(String typeString, ClassLoader cl) { try { - return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_OUT_1, cl); + return InstantiationUtil.readObjectFromConfig(this.config, typeString == null + ? TYPE_SERIALIZER_OUT_1 : TYPE_SERIALIZER_SIDEOUT_PREFIX + typeString, cl); } catch (Exception e) { throw new StreamTaskException("Could not instantiate serializer.", e); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java index 6d7722c60960d..e4d3a089daa18 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java @@ -21,7 +21,7 @@ import java.util.List; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; /** @@ -44,23 +44,22 @@ public class StreamEdge implements Serializable { */ final private int typeNumber; - private final TypeInformation outputType; - /** * A list of output names that the target vertex listens to (if there is * output selection). */ private final List selectedNames; + private final OutputTag outputTag; private StreamPartitioner outputPartitioner; public StreamEdge(StreamNode sourceVertex, StreamNode targetVertex, int typeNumber, - List selectedNames, StreamPartitioner outputPartitioner, TypeInformation outputType) { + List selectedNames, StreamPartitioner outputPartitioner, OutputTag outputTag) { this.sourceVertex = sourceVertex; this.targetVertex = targetVertex; this.typeNumber = typeNumber; this.selectedNames = selectedNames; this.outputPartitioner = outputPartitioner; - this.outputType = outputType; + this.outputTag = outputTag; this.edgeId = sourceVertex + "_" + targetVertex + "_" + typeNumber + "_" + selectedNames + "_" + outputPartitioner; @@ -90,7 +89,9 @@ public List getSelectedNames() { return selectedNames; } - public TypeInformation getOutputType() { return outputType; } + public String getSideOutputTypeName() { return outputTag == null ? null : outputTag.getTypeInfo().toString();} + + public OutputTag getOutputTag() {return this.outputTag;} public StreamPartitioner getPartitioner() { return outputPartitioner; @@ -123,6 +124,6 @@ public boolean equals(Object o) { public String toString() { return "(" + sourceVertex + " -> " + targetVertex + ", typeNumber=" + typeNumber + ", selectedNames=" + selectedNames + ", outputPartitioner=" + outputPartitioner - + ')'; + + ", outputTag=" + outputTag + ')'; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index ab9a758043e70..a48b3fcaf1875 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -32,8 +32,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.OutputTagUtil; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; @@ -86,6 +86,7 @@ public class StreamGraph extends StreamingPlan { private Set sources; private Set sinks; private Map>> virtualSelectNodes; + private Map> virtualOutputNodes; private Map>> virtuaPartitionNodes; protected Map vertexIDtoBrokerID; @@ -109,6 +110,7 @@ public StreamGraph(StreamExecutionEnvironment environment) { public void clear() { streamNodes = new HashMap<>(); virtualSelectNodes = new HashMap<>(); + virtualOutputNodes = new HashMap<>(); virtuaPartitionNodes = new HashMap<>(); vertexIDtoBrokerID = new HashMap<>(); vertexIDtoLoopTimeout = new HashMap<>(); @@ -292,6 +294,21 @@ public void addVirtualSelectNode(Integer originalId, Integer virtualId, List>(originalId, selectedNames)); } + /** + * Adds a new virtual node that is used to connect a downstream vertex to only outputs with OutputTag + * @param originalId ID of the node that should be connected to. + * @param virtualId ID of the virtual node. + * @param outputTag assigned OutputTag to elements + */ + public void addVirtualOutputNode(Integer originalId, Integer virtualId, OutputTag outputTag) { + + if (virtualOutputNodes.containsKey(virtualId)) { + throw new IllegalStateException("Already has virtual output node with id " + virtualId); + } + + virtualOutputNodes.put(virtualId, new Tuple2<>(originalId, outputTag)); + } + /** * Adds a new virtual node that is used to connect a downstream vertex to an input with a certain * partitioning. @@ -317,7 +334,10 @@ public void addVirtualPartitionNode(Integer originalId, Integer virtualId, Strea * Determines the slot sharing group of an operation across virtual nodes. */ public String getSlotSharingGroup(Integer id) { - if (virtualSelectNodes.containsKey(id)) { + if (virtualOutputNodes.containsKey(id)) { + Integer mappedId = virtualOutputNodes.get(id).f0; + return getSlotSharingGroup(mappedId); + } else if (virtualSelectNodes.containsKey(id)) { Integer mappedId = virtualSelectNodes.get(id).f0; return getSlotSharingGroup(mappedId); } else if (virtuaPartitionNodes.containsKey(id)) { @@ -334,7 +354,7 @@ public void addEdge(Integer upStreamVertexID, Integer downStreamVertexID, int ty downStreamVertexID, typeNumber, null, - new ArrayList()); + new ArrayList(), null); } @@ -342,17 +362,25 @@ private void addEdgeInternal(Integer upStreamVertexID, Integer downStreamVertexID, int typeNumber, StreamPartitioner partitioner, - List outputNames) { - + List outputNames, + OutputTag outputTag) { - if (virtualSelectNodes.containsKey(upStreamVertexID)) { + if (virtualOutputNodes.containsKey(upStreamVertexID)) { + int virtualId = upStreamVertexID; + upStreamVertexID = virtualOutputNodes.get(virtualId).f0; + if (outputTag == null) { + // selections that happen downstream override earlier selections + outputTag = virtualOutputNodes.get(virtualId).f1; + } + addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, null, outputTag); + }else if (virtualSelectNodes.containsKey(upStreamVertexID)) { int virtualId = upStreamVertexID; upStreamVertexID = virtualSelectNodes.get(virtualId).f0; if (outputNames.isEmpty()) { // selections that happen downstream override earlier selections outputNames = virtualSelectNodes.get(virtualId).f1; } - addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames); + addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames, null); } else if (virtuaPartitionNodes.containsKey(upStreamVertexID)) { int virtualId = upStreamVertexID; upStreamVertexID = virtuaPartitionNodes.get(virtualId).f0; @@ -371,7 +399,7 @@ private void addEdgeInternal(Integer upStreamVertexID, configurableStreamPartitioner.configure(downstreamNode.getMaxParallelism()); } - addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames); + addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames, outputTag); } else { StreamNode upstreamNode = getStreamNode(upStreamVertexID); StreamNode downstreamNode = getStreamNode(downStreamVertexID); @@ -393,7 +421,7 @@ private void addEdgeInternal(Integer upStreamVertexID, } } - StreamEdge edge = new StreamEdge(upstreamNode, downstreamNode, typeNumber, outputNames, partitioner, OutputTagUtil.getSideOutputTypeInfo(outputNames)); + StreamEdge edge = new StreamEdge(upstreamNode, downstreamNode, typeNumber, outputNames, partitioner, outputTag); getStreamNode(edge.getSourceId()).addOutEdge(edge); getStreamNode(edge.getTargetId()).addInEdge(edge); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java index e49d21e3b9253..8ccaa2ad68823 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.api.graph; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.OutputTagUtil; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; @@ -310,10 +309,10 @@ private Collection transformSelect(SelectTransformation select) } /** - * Transforms a {@code SelectTransformation}. + * Transforms a {@code SideOutputTransformation}. * *

- * For this we create a virtual node in the {@code StreamGraph} holds the selected names. + * For this we create a virtual node in the {@code StreamGraph} holds the output node accepts one outputtag. * @see org.apache.flink.streaming.api.graph.StreamGraphGenerator */ private Collection transformSideOutput(SideOutputTransformation sideOutput) { @@ -330,7 +329,7 @@ private Collection transformSideOutput(SideOutputTransformation for (int inputId : resultIds) { int virtualId = StreamTransformation.getNewNodeId(); - streamGraph.addVirtualSelectNode(inputId, virtualId, OutputTagUtil.getOutputTagName(sideOutput.getOutputTag())); + streamGraph.addVirtualOutputNode(inputId, virtualId, sideOutput.getOutputTag()); virtualResultIds.add(virtualId); } return virtualResultIds; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java index 84e1f19ab8341..4d62db8593bf0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java @@ -17,6 +17,12 @@ package org.apache.flink.streaming.api.graph; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; @@ -29,12 +35,6 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.util.Preconditions; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Class representing the operators in the streaming programs, with all their properties. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 48be2e982da9a..0c1eb9ba1b730 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.api.graph; +import com.google.common.collect.Iterables; import com.google.common.hash.HashFunction; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; @@ -334,6 +335,16 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, config.setTypeSerializerIn2(vertex.getTypeSerializerIn2()); config.setTypeSerializerOut(vertex.getTypeSerializerOut()); + // iterate edges, find sideOutput edges create and save serializers for each outputTag type + for(StreamEdge edge : Iterables.concat(nonChainableOutputs, chainableOutputs)) { + if(edge.getOutputTag() != null) { + config.setTypeSerializerSideOuts( + edge.getSideOutputTypeName(), + edge.getOutputTag().getTypeInfo().createSerializer(streamGraph.getExecutionConfig()) + ); + } + } + config.setStreamOperator(vertex.getOperator()); config.setOutputSelectors(vertex.getOutputSelectors()); @@ -429,6 +440,7 @@ private boolean isChainable(StreamEdge edge) { headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) && (edge.getPartitioner() instanceof ForwardPartitioner) && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() + && edge.getOutputTag() == null //disable chain for sideouput && streamGraph.isChainingEnabled(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java index e5843c88a860b..a6132a6822d2a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java @@ -17,6 +17,8 @@ */ package org.apache.flink.streaming.api.transformations; +import static java.util.Objects.requireNonNull; + import com.google.common.collect.Lists; import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -24,6 +26,15 @@ import java.util.Collection; import java.util.List; + +/** + * This transformation represents a selection of only upstream elements with one outputTag. + *

+ * This does not create a physical operation, it only affects how upstream operations are + * connected to downstream operations. + * + * @param The type of the elements that result from this {@code SideOutputTransformation} + */ public class SideOutputTransformation extends StreamTransformation { private final StreamTransformation input; private final OutputTag tag; @@ -31,7 +42,7 @@ public class SideOutputTransformation extends StreamTransformation { public SideOutputTransformation(StreamTransformation input, final OutputTag tag) { super("SideOutput", tag.getTypeInfo(), input.getParallelism()); this.input = input; - this.tag = tag; + this.tag = requireNonNull(tag); } /** @@ -55,6 +66,6 @@ public Collection> getTransitivePredecessors() { @Override public final void setChainingStrategy(ChainingStrategy strategy) { - throw new UnsupportedOperationException("Cannot set chaining strategy on Select Transformation."); + throw new UnsupportedOperationException("Cannot set chaining strategy on SideOutput Transformation."); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index 40632c4bc084a..4a8cf6ab1e748 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; @@ -43,15 +44,16 @@ public class RecordWriterOutput implements Output> { private SerializationDelegate serializationDelegate; - private TypeSerializer outRecordSerializer; + private final OutputTag tag; @SuppressWarnings("unchecked") public RecordWriterOutput( StreamRecordWriter>> recordWriter, + OutputTag tag, TypeSerializer outSerializer) { checkNotNull(recordWriter); - + this.tag = tag; // generic hack: cast the writer to generic Object type so we can use it // with multiplexed records and watermarks this.recordWriter = (StreamRecordWriter>) @@ -67,15 +69,15 @@ public RecordWriterOutput( @Override public void collect(StreamRecord record) { - try { - serializationDelegate.setInstance(record); - recordWriter.emit(serializationDelegate); + if((record.getOutputTag() == null && this.tag == null) + || (record.getOutputTag() != null && record.getOutputTag().equals(this.tag))) { + serializationDelegate.setInstance(record); + recordWriter.emit(serializationDelegate); + } } catch (Exception e) { - //Hack, should do type check - //throw new RuntimeException(e.getMessage(), e); + throw new RuntimeException(e.getMessage(), e); } - } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java index edcd8339c306e..877235a15ba80 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java @@ -51,6 +51,7 @@ import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import java.io.Serializable; import java.util.Collection; @@ -220,6 +221,9 @@ public void processElement(StreamRecord element) throws Exception { Collection elementWindows = windowAssigner.assignWindows( element.getValue(), element.getTimestamp(), windowAssignerContext); + //if element is handled by none of assigned elementWindows + boolean isSkippedElement = true; + final K key = (K) getKeyedStateBackend().getCurrentKey(); if (windowAssigner instanceof MergingWindowAssigner) { @@ -260,6 +264,7 @@ public void merge(W mergeResult, mergingWindows.retireWindow(actualWindow); continue; } + isSkippedElement = false; W stateWindow = mergingWindows.getStateWindow(actualWindow); if (stateWindow == null) { @@ -299,6 +304,7 @@ public void merge(W mergeResult, if (isLate(window)) { continue; } + isSkippedElement = false; AppendingState windowState = getPartitionedState(window, windowSerializer, windowStateDescriptor); @@ -324,6 +330,10 @@ public void merge(W mergeResult, } } } + + if(isSkippedElement && isLate(element)) { + sideOutput(element); + } } @Override @@ -431,6 +441,14 @@ private void fire(W window, ACC contents) throws Exception { userFunction.apply(context.key, context.window, contents, timestampedCollector); } + /** + * write skipped late arriving element to SideOutput + * @param element element to side output + */ + private void sideOutput(StreamRecord element){ + timestampedCollector.collect(new LateArrivingOutputTag(), element); + } + /** * Retrieves the {@link MergingWindowSet} for the currently active key. * The caller must ensure that the correct key is set in the state backend. @@ -460,6 +478,16 @@ protected boolean isLate(W window) { return (windowAssigner.isEventTime() && (cleanupTime(window) <= internalTimerService.currentWatermark())); } + /** + * Decide if a record is currently late, based on current watermark and allowed lateness + * @param element The element to check + * @return The element for which should be considered when sideoutput + */ + protected boolean isLate(StreamRecord element){ + return (windowAssigner.isEventTime()) && + (element.getTimestamp() + allowedLateness <= internalTimerService.currentWatermark()); + } + /** * Registers a timer to cleanup the content of the window. * @param window diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java index 164186b55c15d..474d148693201 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java @@ -17,6 +17,8 @@ package org.apache.flink.streaming.runtime.streamrecord; +import static java.util.Objects.requireNonNull; + import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -47,7 +49,14 @@ public StreamRecord(T value) { this.value = value; } - public TypeInformation getType() { return this.tag == null ? null : this.tag.getTypeInfo();} + public StreamRecord(T value, OutputTag tag) { + this.value = value; + this.tag = requireNonNull(tag); + } + + public TypeInformation getType() { return tag == null ? null : tag.getTypeInfo();} + + public OutputTag getOutputTag() { return tag; } /** * Creates a new StreamRecord wrapping the given value. The timestamp is set to the @@ -58,10 +67,27 @@ public StreamRecord(T value) { */ public StreamRecord(T value, long timestamp) { this.value = value; + this.tag = null; this.timestamp = timestamp; this.hasTimestamp = true; } + /** + * Creates a new StreamRecord wrapping the given value. The timestamp is set to the + * given timestamp, OutputTag is set to given sideOutputTag + * + * @param value The value to wrap in this {@link StreamRecord} + * @param timestamp The timestamp in milliseconds + * @param tag The sideoutputTag in this {@link StreamRecord} + * + */ + public StreamRecord(T value, long timestamp, OutputTag tag){ + this.value = value; + this.timestamp = timestamp; + this.hasTimestamp = true; + this.tag = requireNonNull(tag); + } + // ------------------------------------------------------------------------ // Accessors // ------------------------------------------------------------------------ @@ -102,7 +128,7 @@ public boolean hasTimestamp() { /** * Replace the currently stored value by the given new value. This returns a StreamElement * with the generic type parameter that matches the new value while keeping the old - * timestamp. + * timestamp. Clean outputTag if any * * @param element Element to set in this stream value * @return Returns the StreamElement with replaced value @@ -110,12 +136,23 @@ public boolean hasTimestamp() { @SuppressWarnings("unchecked") public StreamRecord replace(X element) { this.value = (T) element; + this.tag = null; return (StreamRecord) this; } + /** + * Replace the currently stored value by the given new value. This returns a StreamElement + * with the generic type parameter that matches the new value while keeping the old + * timestamp. Overwrite outputTag if any + * @param tag outputTag of Element + * @param element Element to set in this stream value + * @param Returns the StreamElement with replaced value and outputTag + * @return + */ @SuppressWarnings("unchecked") public StreamRecord replace(OutputTag tag, X element) { this.value = (T) element; + this.tag = tag; return (StreamRecord) this; } @@ -132,6 +169,7 @@ public StreamRecord replace(OutputTag tag, X element) { public StreamRecord replace(X value, long timestamp) { this.timestamp = timestamp; this.value = (T) value; + this.tag = null; this.hasTimestamp = true; return (StreamRecord) this; @@ -156,6 +194,7 @@ public void eraseTimestamp() { */ public StreamRecord copy(T valueCopy) { StreamRecord copy = new StreamRecord<>(valueCopy); + copy.tag = this.tag; copy.timestamp = this.timestamp; copy.hasTimestamp = this.hasTimestamp; return copy; @@ -167,6 +206,7 @@ public StreamRecord copy(T valueCopy) { */ public void copyTo(T valueCopy, StreamRecord target) { target.value = valueCopy; + target.tag = this.tag; target.timestamp = this.timestamp; target.hasTimestamp = this.hasTimestamp; } @@ -184,7 +224,8 @@ else if (o != null && getClass() == o.getClass()) { StreamRecord that = (StreamRecord) o; return this.hasTimestamp == that.hasTimestamp && (!this.hasTimestamp || this.timestamp == that.timestamp) && - (this.value == null ? that.value == null : this.value.equals(that.value)); + (this.value == null ? that.value == null : this.value.equals(that.value)) && + (this.tag == null ? that.tag == null : this.tag.equals(that.tag)); } else { return false; @@ -194,11 +235,14 @@ else if (o != null && getClass() == o.getClass()) { @Override public int hashCode() { int result = value != null ? value.hashCode() : 0; - return 31 * result + (hasTimestamp ? (int) (timestamp ^ (timestamp >>> 32)) : 0); + result = 31 * result + (hasTimestamp ? (int) (timestamp ^ (timestamp >>> 32)) : 0); + result = 31 * result + (tag != null ? (int) (tag.hashCode() ^ (tag.hashCode() >>> 32)) : 0); + return result; } @Override public String toString() { - return "Record @ " + (hasTimestamp ? timestamp : "(undef)") + " : " + value; + return "Record @ " + (hasTimestamp ? timestamp : "(undef)") + " : " + value + "#" + + (tag == null ? "notag" : tag.toString()); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index 80a1e7226f32d..84e4477e03e04 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -17,7 +17,15 @@ package org.apache.flink.streaming.runtime.tasks; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.metrics.Counter; @@ -45,13 +53,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; - /** * The {@code OperatorChain} contains all operators that are executed as one chain within a single @@ -323,7 +324,9 @@ private static RecordWriterOutput createStreamOutput( StreamEdge edge, StreamConfig upStreamConfig, int outputIndex, Environment taskEnvironment, String taskName) { - TypeSerializer outSerializer = upStreamConfig.getTypeSerializerOut(taskEnvironment.getUserClassLoader()); + OutputTag sideOutputTag = edge.getOutputTag(); // OutputTag, return null if not sideOutput + TypeSerializer outSerializer = upStreamConfig.getTypeSerializerOut(edge.getSideOutputTypeName(), + taskEnvironment.getUserClassLoader()); @SuppressWarnings("unchecked") StreamPartitioner outputPartitioner = (StreamPartitioner) edge.getPartitioner(); @@ -336,7 +339,7 @@ private static RecordWriterOutput createStreamOutput( new StreamRecordWriter<>(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout()); output.setMetricGroup(taskEnvironment.getMetricGroup().getIOMetricGroup()); - return new RecordWriterOutput<>(output, outSerializer); + return new RecordWriterOutput<>(output, sideOutputTag, outSerializer); } // ------------------------------------------------------------------------ @@ -407,18 +410,16 @@ public CopyingChainingOutput(OneInputStreamOperator operator, TypeSerializ @Override public void collect(StreamRecord record) { - //Hack, shoud do better type check - if(record.getValue().getClass() == serializer.createInstance().getClass()) { - try { + try { + // sideOutput still not support chainning yet, filter out sideoutputs + if (record.getOutputTag() == null) { numRecordsIn.inc(); StreamRecord copy = record.copy(serializer.copy(record.getValue())); operator.setKeyContextElement1(copy); operator.processElement(copy); - } catch (Exception e) { - throw new RuntimeException("Could not forward element to next operator", e); } - } else { - LOG.info("{} {}",record.getValue().getClass().toString(), serializer.createInstance().getClass().toString()); + } catch (Exception e) { + throw new RuntimeException("Could not forward element to next operator", e); } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java new file mode 100644 index 0000000000000..22b475cd3b736 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java @@ -0,0 +1,34 @@ +/** + * 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.util.outputtags; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.OutputTag; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +/** + * {@code OutputTag} for late arriving events + * @param {@code WindowOperator} input type + */ +@Internal +public final class LateArrivingOutputTag extends OutputTag>{ + + public LateArrivingOutputTag(){ + super(); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java index e4e29c1d627fd..cc55d80fa712e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java @@ -21,12 +21,14 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.util.Collector; +import org.apache.flink.util.CollectorWrapper; import org.junit.Assert; import org.junit.Test; @@ -37,10 +39,13 @@ *

  • RichFunction methods are called correctly
  • *
  • Timestamps of processed elements match the input timestamp
  • *
  • Watermarks are correctly forwarded
  • + *
  • sideOutput are correctly processed
  • * */ public class StreamFlatMapTest { + private static final OutputTag sideOutputTag = new OutputTag() {}; + public static final class MyFlatMap implements FlatMapFunction { private static final long serialVersionUID = 1L; @@ -104,6 +109,7 @@ public void testOpenClose() throws Exception { Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseFlatMapFunction.closeCalled); Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + Assert.assertTrue("SideOutput contains no elements.", testHarness.getSideOutput(sideOutputTag).size() > 0); } // This must only be used in one test, otherwise the static fields will be changed @@ -138,6 +144,9 @@ public void flatMap(String value, Collector out) throws Exception { Assert.fail("Open was not called before run."); } out.collect(value); + + CollectorWrapper wrapper = new CollectorWrapper<>(out); + wrapper.collect(sideOutputTag, 1); } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java index 4dbf7b8d03d2c..79fa0a434cc43 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java @@ -22,6 +22,7 @@ import java.io.Serializable; import java.util.concurrent.ConcurrentLinkedQueue; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; @@ -30,6 +31,7 @@ import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness; import org.apache.flink.util.Collector; +import org.apache.flink.util.CollectorWrapper; import org.junit.Assert; import org.junit.Test; @@ -40,11 +42,14 @@ *
  • RichFunction methods are called correctly
  • *
  • Timestamps of processed elements match the input timestamp
  • *
  • Watermarks are correctly forwarded
  • + *
  • sideOutput are correctly processed
  • * */ public class CoStreamFlatMapTest implements Serializable { private static final long serialVersionUID = 1L; + private static final OutputTag sideOutputTag = new OutputTag() {}; + private final static class MyCoFlatMap implements CoFlatMapFunction { private static final long serialVersionUID = 1L; @@ -121,6 +126,7 @@ public void testOpenClose() throws Exception { Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseCoFlatMapFunction.closeCalled); Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + Assert.assertTrue("SideOutput contains no elements.", testHarness.getSideOutput(sideOutputTag).size() > 0); } // This must only be used in one test, otherwise the static fields will be changed @@ -163,6 +169,8 @@ public void flatMap2(Integer value, Collector out) throws Exception { Assert.fail("Open was not called before run."); } out.collect(value.toString()); + CollectorWrapper wrapper = new CollectorWrapper<>(out); + wrapper.collect(sideOutputTag, 1); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index 0e2d1e8e29783..2eab37e25dba0 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -67,6 +67,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.WindowingTestHarness; +import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -88,6 +89,9 @@ public class WindowOperatorTest extends TestLogger { // For counting if close() is called the correct number of times on the SumReducer private static AtomicInteger closeCalled = new AtomicInteger(0); + //late arriving event OutputTag> + private static final LateArrivingOutputTag tag = new LateArrivingOutputTag>(); + private void testSlidingEventTimeWindows(OneInputStreamOperatorTestHarness, Tuple2> testHarness) throws Exception { ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); @@ -1129,6 +1133,7 @@ public void testLateness() throws Exception { testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue lateExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 500)); testHarness.processWatermark(new Watermark(1500)); @@ -1141,7 +1146,7 @@ public void testLateness() throws Exception { expected.add(new StreamRecord<>(new Tuple2<>("key2", 2), 1999)); expected.add(new Watermark(2300)); - // this will not be dropped because window.maxTimestamp() + allowedLateness > currentWatermark + // this will not be sideoutput because window.maxTimestamp() + allowedLateness > currentWatermark testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1997)); testHarness.processWatermark(new Watermark(6000)); @@ -1149,13 +1154,15 @@ public void testLateness() throws Exception { expected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1999)); expected.add(new Watermark(6000)); - // this will be dropped because window.maxTimestamp() + allowedLateness < currentWatermark + // this will be sideoutput because window.maxTimestamp() + allowedLateness < currentWatermark testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998)); testHarness.processWatermark(new Watermark(7000)); + lateExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1998, tag)); expected.add(new Watermark(7000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", lateExpected, testHarness.getSideOutput(tag), new Tuple2ResultSortComparator()); testHarness.close(); } @@ -1228,7 +1235,7 @@ public long getCurrentProcessingTime() { } @Test - public void testDropDueToLatenessTumbling() throws Exception { + public void testSideOutputDueToLatenessTumbling() throws Exception { final int WINDOW_SIZE = 2; final long LATENESS = 0; @@ -1255,6 +1262,7 @@ public void testDropDueToLatenessTumbling() throws Exception { testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); // normal element testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); @@ -1269,8 +1277,9 @@ public void testDropDueToLatenessTumbling() throws Exception { expected.add(new StreamRecord<>(new Tuple2<>("key2", 2), 1999)); expected.add(new Watermark(1999)); - // dropped as late + // sideoutput as late testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1998, tag)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 2001)); testHarness.processWatermark(new Watermark(2999)); @@ -1283,11 +1292,12 @@ public void testDropDueToLatenessTumbling() throws Exception { expected.add(new Watermark(3999)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, testHarness.getSideOutput(tag), new Tuple2ResultSortComparator()); testHarness.close(); } @Test - public void testDropDueToLatenessSliding() throws Exception { + public void testSideOutputDueToLatenessSliding() throws Exception { final int WINDOW_SIZE = 3; final int WINDOW_SLIDE = 1; final long LATENESS = 0; @@ -1315,6 +1325,7 @@ public void testDropDueToLatenessSliding() throws Exception { testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1351,19 +1362,21 @@ public void testDropDueToLatenessSliding() throws Exception { expected.add(new Watermark(6000)); - // dropped due to lateness + // sideoutput element due to lateness testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 3001)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 3001, tag)); testHarness.processWatermark(new Watermark(25000)); expected.add(new Watermark(25000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, testHarness.getSideOutput(tag), new Tuple2ResultSortComparator()); testHarness.close(); } @Test - public void testDropDueToLatenessSessionZeroLatenessPurgingTrigger() throws Exception { + public void testSideOutputDueToLatenessSessionZeroLatenessPurgingTrigger() throws Exception { final int GAP_SIZE = 3; final long LATENESS = 0; @@ -1390,6 +1403,7 @@ public void testDropDueToLatenessSessionZeroLatenessPurgingTrigger() throws Exce testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1425,10 +1439,13 @@ public void testDropDueToLatenessSessionZeroLatenessPurgingTrigger() throws Exce expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 11600L, 14600L), 14599)); expected.add(new Watermark(14600)); - // this is dropped as late + // this is sideoutput as late testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000)); - // this is also dropped as late (we test that they are not accidentally merged) + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10000, tag)); + + // this is also sideoutput as late (we test that they are not accidentally merged) testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10100)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10100, tag)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1441,14 +1458,17 @@ public void testDropDueToLatenessSessionZeroLatenessPurgingTrigger() throws Exce expected.add(new Watermark(100000)); ConcurrentLinkedQueue actual = testHarness.getOutput(); + ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); + testHarness.close(); } @Test - public void testDropDueToLatenessSessionZeroLateness() throws Exception { - // same as testDropDueToLatenessSessionZeroLateness() but with an accumulating trigger, i.e. + public void testSideOutputDueToLatenessSessionZeroLateness() throws Exception { + // same as testSideOutputDueToLatenessSessionZeroLateness() but with an accumulating trigger, i.e. // one that does not return FIRE_AND_PURGE when firing but just FIRE // this has the same output as testDropDueToLatenessSessionZeroLateness() because @@ -1480,6 +1500,7 @@ public void testDropDueToLatenessSessionZeroLateness() throws Exception { testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1515,8 +1536,9 @@ public void testDropDueToLatenessSessionZeroLateness() throws Exception { expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 11600L, 14600L), 14599)); expected.add(new Watermark(14600)); - // this is dropped as late + // this is sideoutput as late testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10000, tag)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1528,12 +1550,14 @@ public void testDropDueToLatenessSessionZeroLateness() throws Exception { expected.add(new Watermark(100000)); ConcurrentLinkedQueue actual = testHarness.getOutput(); + ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); testHarness.close(); } @Test - public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exception { + public void testSideOutputDueToLatenessSessionWithLatenessPurgingTrigger() throws Exception { // this has the same output as testDropDueToLatenessSessionZeroLateness() because // the allowed lateness is too small to make a difference @@ -1564,6 +1588,7 @@ public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exce testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1599,8 +1624,9 @@ public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exce expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 11600L, 14600L), 14599)); expected.add(new Watermark(14600)); - // dropped as late + // sideoutput as late testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10000, tag)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1612,13 +1638,15 @@ public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exce expected.add(new Watermark(100000)); ConcurrentLinkedQueue actual = testHarness.getOutput(); + ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); testHarness.close(); } @Test - public void testDropDueToLatenessSessionWithLateness() throws Exception { - // same as testDropDueToLatenessSessionWithLateness() but with an accumulating trigger, i.e. + public void testSideOutputDueToLatenessSessionWithLateness() throws Exception { + // same as testSideOutputDueToLatenessSessionWithLateness() but with an accumulating trigger, i.e. // one that does not return FIRE_AND_PURGE when firing but just FIRE. The expected // results are therefore slightly different. @@ -1648,6 +1676,7 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception { testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1659,7 +1688,7 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception { expected.add(new Watermark(4998)); - // this will not be dropped because the session we're adding two has maxTimestamp + // this will not be sideoutput because the session we're adding two has maxTimestamp // after the current watermark testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 4500)); @@ -1685,7 +1714,7 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception { // because of the small allowed lateness and because the trigger is accumulating // this will be merged into the session (11600-14600) and therefore will not - // be dropped as late + // be sideoutput as late testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); @@ -1696,7 +1725,10 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception { expected.add(new StreamRecord<>(new Tuple3<>("key2-2", 10000L, 14600L), 14599)); ConcurrentLinkedQueue actual = testHarness.getOutput(); + ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); testHarness.processWatermark(new Watermark(20000)); @@ -1708,12 +1740,15 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception { expected.add(new Watermark(100000)); actual = testHarness.getOutput(); + sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); + testHarness.close(); } @Test - public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws Exception { + public void testSideOutputDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws Exception { final int GAP_SIZE = 3; final long LATENESS = 10000; @@ -1741,6 +1776,7 @@ public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1752,7 +1788,7 @@ public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws expected.add(new Watermark(4998)); - // this will not be dropped because the session we're adding two has maxTimestamp + // this will not be sideoutput because the session we're adding two has maxTimestamp // after the current watermark testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 4500)); @@ -1781,7 +1817,9 @@ public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 10000L, 13000L), 12999)); ConcurrentLinkedQueue actual = testHarness.getOutput(); + ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1794,12 +1832,14 @@ public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws expected.add(new Watermark(100000)); actual = testHarness.getOutput(); + sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); testHarness.close(); } @Test - public void testDropDueToLatenessSessionWithHugeLateness() throws Exception { + public void testSideOutputDueToLatenessSessionWithHugeLateness() throws Exception { final int GAP_SIZE = 3; final long LATENESS = 10000; @@ -1826,6 +1866,7 @@ public void testDropDueToLatenessSessionWithHugeLateness() throws Exception { testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1837,7 +1878,7 @@ public void testDropDueToLatenessSessionWithHugeLateness() throws Exception { expected.add(new Watermark(4998)); - // this will not be dropped because the session we're adding two has maxTimestamp + // this will not be sideoutput because the session we're adding two has maxTimestamp // after the current watermark testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 4500)); @@ -1868,7 +1909,9 @@ public void testDropDueToLatenessSessionWithHugeLateness() throws Exception { expected.add(new StreamRecord<>(new Tuple3<>("key2-7", 1000L, 14600L), 14599)); ConcurrentLinkedQueue actual = testHarness.getOutput(); + ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1880,7 +1923,11 @@ public void testDropDueToLatenessSessionWithHugeLateness() throws Exception { expected.add(new Watermark(100000)); actual = testHarness.getOutput(); + sideActual = testHarness.getSideOutput(tag); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); + testHarness.close(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordTest.java index 5d34b74f08bff..c0a209e5152a0 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.streamrecord; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.junit.Test; import static org.junit.Assert.*; @@ -155,4 +156,31 @@ public void testSetAndEraseTimestamps() { rec.eraseTimestamp(); assertFalse(rec.hasTimestamp()); } + + @Test + public void testReplaceWithOutputTag() { + OutputTag sideOutputTag = new OutputTag() {}; + + StreamRecord rec = new StreamRecord(null); + rec.setTimestamp(13456L); + + assertEquals(null, rec.getOutputTag()); + assertEquals(13456L, rec.getTimestamp()); + + rec.replace(sideOutputTag, "hello"); + assertEquals(sideOutputTag, rec.getOutputTag()); + assertEquals(13456L, rec.getTimestamp()); + + rec.replace("word"); + assertEquals(null, rec.getOutputTag()); + assertEquals(13456L, rec.getTimestamp()); + } + + @Test + public void testCopyToWithOutputTag() { + StreamRecord recWithTimestamp = new StreamRecord("test", 99, new OutputTag() {}); + StreamRecord recWithTimestampCopy = new StreamRecord<>(null); + recWithTimestamp.copyTo("test", recWithTimestampCopy); + assertEquals(recWithTimestamp, recWithTimestampCopy); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java index 830cd6faadb0e..de541569e3959 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java @@ -19,6 +19,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.Configuration; @@ -78,6 +79,8 @@ public class AbstractStreamOperatorTestHarness { final protected ConcurrentLinkedQueue outputList; + final protected ConcurrentLinkedQueue sideOutputList; + final protected StreamConfig config; final protected ExecutionConfig executionConfig; @@ -113,6 +116,7 @@ public AbstractStreamOperatorTestHarness( int subtaskIndex) throws Exception { this.operator = operator; this.outputList = new ConcurrentLinkedQueue<>(); + this.sideOutputList = new ConcurrentLinkedQueue<>(); Configuration underlyingConfig = new Configuration(); this.config = new StreamConfig(underlyingConfig); this.config.setCheckpointingEnabled(true); @@ -216,6 +220,17 @@ public ConcurrentLinkedQueue getOutput() { return outputList; } + //TODO: FIX ME + public ConcurrentLinkedQueue getSideOutput(OutputTag tag) { + ConcurrentLinkedQueue matchList = new ConcurrentLinkedQueue(); + for(Object item : sideOutputList){ + if(item instanceof StreamRecord && tag.equals(((StreamRecord)item).getOutputTag())){ + matchList.add(item); + } + } + return matchList; + } + /** * Get only the {@link StreamRecord StreamRecords} emitted by the operator. */ @@ -503,6 +518,8 @@ private class MockOutput implements Output> { private TypeSerializer outputSerializer; + private TypeSerializer sideOutputSerializer; + MockOutput() { this(null); } @@ -523,13 +540,25 @@ public void emitLatencyMarker(LatencyMarker latencyMarker) { @Override public void collect(StreamRecord element) { - if (outputSerializer == null) { - outputSerializer = TypeExtractor.getForObject(element.getValue()).createSerializer(executionConfig); - } - if (element.hasTimestamp()) { - outputList.add(new StreamRecord<>(outputSerializer.copy(element.getValue()),element.getTimestamp())); + if (element.getOutputTag() != null) { + StreamRecord record = element.getValue() instanceof StreamRecord ? (StreamRecord) element.getValue() : element; + if (sideOutputSerializer == null) { + sideOutputSerializer = TypeExtractor.getForObject(record.getValue()).createSerializer(executionConfig); + } + if (element.hasTimestamp()) { + sideOutputList.add(new StreamRecord<>(sideOutputSerializer.copy(record.getValue()), record.getTimestamp(), element.getOutputTag())); + } else { + sideOutputList.add(new StreamRecord<>(sideOutputSerializer.copy(record.getValue()), element.getOutputTag())); + } } else { - outputList.add(new StreamRecord<>(outputSerializer.copy(element.getValue()))); + if (outputSerializer == null) { + outputSerializer = TypeExtractor.getForObject(element.getValue()).createSerializer(executionConfig); + } + if (element.hasTimestamp()) { + outputList.add(new StreamRecord<>(outputSerializer.copy(element.getValue()), element.getTimestamp())); + } else { + outputList.add(new StreamRecord<>(outputSerializer.copy(element.getValue()))); + } } } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala index 83104e8318f8f..d04ac8c7c492e 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala @@ -18,18 +18,21 @@ package org.apache.flink.streaming.api.scala -import org.apache.flink.annotation.{PublicEvolving, Public} +import org.apache.flink.annotation.{Public, PublicEvolving} import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWStream} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} +import org.apache.flink.streaming.api.functions.windowing.{DiscardAllWindowFunction} import org.apache.flink.streaming.api.scala.function.AllWindowFunction -import org.apache.flink.streaming.api.scala.function.util.{ScalaAllWindowFunction, ScalaAllWindowFunctionWrapper, ScalaReduceFunction, ScalaFoldFunction} +import org.apache.flink.streaming.api.scala.function.util.{ScalaAllWindowFunction, ScalaAllWindowFunctionWrapper, ScalaFoldFunction, ScalaReduceFunction} import org.apache.flink.streaming.api.windowing.evictors.Evictor import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.triggers.Trigger import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord +import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag import org.apache.flink.util.Collector /** @@ -465,6 +468,16 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) { asScalaStream(javaStream.apply(initialValue, folder, applyFunction, returnType)) } + /** + * Applies a DiscardWindowFunction that only returns late arriving events + * @return the data stream considered too late to be evaluated by any windows + */ + def tooLateEvents(): DataStream[StreamRecord[T]] = { + val discardAllWindowFunction = new DiscardAllWindowFunction[T, W]() + val lateArrivingOutputTag = new LateArrivingOutputTag[T]() + asScalaStream(javaStream.apply(discardAllWindowFunction).getSideOutput(lateArrivingOutputTag)) + } + // ------------------------------------------------------------------------ // Aggregations on the keyed windows // ------------------------------------------------------------------------ diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 5ad5b5b450af9..1badc2c90a61a 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -202,10 +202,9 @@ class DataStream[T](stream: JavaStream[T]) { } @PublicEvolving - def getOutput(tag: OutputTag[T]): DataStream[T] = javaStream match { - case stream : SingleOutputStreamOperator[T] => asScalaStream(stream.getOutput(tag: OutputTag[T])) - case _ => throw new UnsupportedOperationException("Only supported for operators.") - this + def getSideOutput[X: OutputTag](tag: OutputTag[X]): DataStream[X] = javaStream match { + case stream : SingleOutputStreamOperator[X] => + asScalaStream(stream.getSideOutput(tag: OutputTag[X])) } /** diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala index 76d9cdab0e829..925e8e2e27f5d 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala @@ -18,18 +18,21 @@ package org.apache.flink.streaming.api.scala -import org.apache.flink.annotation.{PublicEvolving, Public} +import org.apache.flink.annotation.{Public, PublicEvolving} import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} +import org.apache.flink.streaming.api.functions.windowing.DiscardWindowFunction import org.apache.flink.streaming.api.scala.function.WindowFunction import org.apache.flink.streaming.api.scala.function.util.{ScalaFoldFunction, ScalaReduceFunction, ScalaWindowFunction, ScalaWindowFunctionWrapper} import org.apache.flink.streaming.api.windowing.evictors.Evictor import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.triggers.Trigger import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord +import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag import org.apache.flink.util.Collector /** @@ -464,6 +467,16 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) { asScalaStream(javaStream.apply(initialValue, folder, applyFunction, resultType)) } + /** + * Applies a DiscardWindowFunction that only returns late arriving events + * @return the data stream considered too late to be evaluated by any windows + */ + def tooLateEvents(): DataStream[StreamRecord[T]] = { + val discardWindowFunction = new DiscardWindowFunction[T, K, W]() + val lateArrivingOutputTag = new LateArrivingOutputTag[T]() + asScalaStream(javaStream.apply(discardWindowFunction).getSideOutput(lateArrivingOutputTag)) + } + // ------------------------------------------------------------------------ // Aggregations on the keyed windows // ------------------------------------------------------------------------ diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java new file mode 100644 index 0000000000000..db15ddf502b51 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java @@ -0,0 +1,204 @@ +/* + * 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.test.streaming.runtime; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.OutputTag; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.datastream.AllWindowedStream; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.datastream.WindowedStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.WindowFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.util.Collector; +import org.apache.flink.util.CollectorWrapper; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.List; + +/** + * Integration test for streaming programs using sideOutputs + * + * See FLINK-4460 + */ +public class SideOutputITCase extends StreamingMultipleProgramsTestBase{ + + static List elements = new ArrayList<>(); + static { + elements.add(1); + elements.add(2); + elements.add(5); + elements.add(3); + elements.add(4); + } + + /** + * Serializable outputTag used as sideOutput + */ + static class SideOutputTag extends OutputTag{ + public SideOutputTag(String value){ + super(value); + } + } + + /** + * Test flatMap sideOutputs + */ + @Test + public void testFlatMapSideOutputs() throws Exception { + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); + see.disableOperatorChaining(); + see.setParallelism(3); + + DataStream dataStream = see.fromCollection(elements); + + SingleOutputStreamOperator passThroughtStream = dataStream.flatMap(new FlatMapFunction() { + + @Override + public void flatMap(Integer value, Collector out) throws Exception { + out.collect(value); + CollectorWrapper wrapper = new CollectorWrapper<>(out); + wrapper.collect(new SideOutputTag("side"), "sideout-" + String.valueOf(value)); + } + }); + + passThroughtStream.getSideOutput(new SideOutputTag("side")).print(); + passThroughtStream.print(); + + passThroughtStream.getSideOutput(new SideOutputTag("notside")).print(); + see.execute(); + } + + /** + * Test flatMap sideOutputs with different outputTag + */ + @Test + public void testFlatMapSideOutputsWithWrongTag() throws Exception { + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); + see.disableOperatorChaining(); + see.setParallelism(3); + + DataStream dataStream = see.fromCollection(elements); + + dataStream.flatMap(new FlatMapFunction() { + + @Override + public void flatMap(Integer value, Collector out) throws Exception { + out.collect(value); + CollectorWrapper wrapper = new CollectorWrapper<>(out); + wrapper.collect(new SideOutputTag("side"), "sideout-" + String.valueOf(value)); + } + }).getSideOutput(new SideOutputTag("notside")).print(); + see.execute(); + } + + private static class TestWatermarkAssigner implements AssignerWithPunctuatedWatermarks{ + @Nullable + @Override + public Watermark checkAndGetNextWatermark(Integer lastElement, long extractedTimestamp) { + return new Watermark(extractedTimestamp); + } + + @Override + public long extractTimestamp(Integer element, long previousElementTimestamp) { + return Long.valueOf(element); + } + } + + private static class TestKeySelector implements KeySelector{ + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + } + + /** + * Test window late arriving events stream + */ + @Test + public void testAllWindowLateArrivingEvents() throws Exception { + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); + see.disableOperatorChaining(); + see.setParallelism(3); + see.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + + DataStream dataStream = see.fromCollection(elements); + AllWindowedStream allWindowedStream = dataStream.assignTimestampsAndWatermarks( + new TestWatermarkAssigner()).timeWindowAll(Time.milliseconds(1), Time.milliseconds(1)); + + allWindowedStream.apply(new AllWindowFunction() { + @Override + public void apply(TimeWindow window, Iterable values, Collector out) throws Exception { + for(Integer val : values) { + out.collect(val); + } + } + }).print(); + + allWindowedStream.tooLateEvents().flatMap(new FlatMapFunction, String>() { + @Override + public void flatMap(StreamRecord value, Collector out) throws Exception { + out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); + } + }).print(); + + see.execute(); + } + + @Test + public void testKeyedWindowLateArrivingEvents() throws Exception { + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); + see.disableOperatorChaining(); + see.setParallelism(3); + see.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + + DataStream dataStream = see.fromCollection(elements); + WindowedStream windowStream = dataStream.assignTimestampsAndWatermarks( + new TestWatermarkAssigner()).keyBy(new TestKeySelector()).timeWindow(Time.milliseconds(1), Time.milliseconds(1)); + + windowStream.apply(new WindowFunction() { + @Override + public void apply(Integer key, TimeWindow window, Iterable input, Collector out) throws Exception { + for(Integer val : input) { + out.collect(String.valueOf(key) + "-"+String.valueOf(val)); + } + } + }).print(); + + windowStream.tooLateEvents().flatMap(new FlatMapFunction, String>() { + @Override + public void flatMap(StreamRecord value, Collector out) throws Exception { + out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); + } + }).print(); + + see.execute(); + } + +} From b28c99c0bf16b4d87ce4add2e422ca326a446aa2 Mon Sep 17 00:00:00 2001 From: Chen Qin Date: Thu, 1 Dec 2016 14:19:56 -0800 Subject: [PATCH 3/5] allow mutiple OutputTag with same type implement windowopeator late arriving events add unit/integration tests --- .../streaming/runtime/SideOutputITCase.java | 61 ++++++++++--------- 1 file changed, 32 insertions(+), 29 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java index db15ddf502b51..df4c1e8e2d609 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java @@ -21,10 +21,8 @@ import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.datastream.AllWindowedStream; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.datastream.WindowedStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; @@ -34,6 +32,7 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import org.apache.flink.util.Collector; import org.apache.flink.util.CollectorWrapper; import org.junit.Test; @@ -149,25 +148,25 @@ public void testAllWindowLateArrivingEvents() throws Exception { see.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); DataStream dataStream = see.fromCollection(elements); - AllWindowedStream allWindowedStream = dataStream.assignTimestampsAndWatermarks( - new TestWatermarkAssigner()).timeWindowAll(Time.milliseconds(1), Time.milliseconds(1)); - allWindowedStream.apply(new AllWindowFunction() { - @Override - public void apply(TimeWindow window, Iterable values, Collector out) throws Exception { - for(Integer val : values) { - out.collect(val); + SingleOutputStreamOperator outputStreamOperator = dataStream.assignTimestampsAndWatermarks( + new TestWatermarkAssigner()).timeWindowAll(Time.milliseconds(1), Time.milliseconds(1)) + .apply(new AllWindowFunction() { + @Override + public void apply(TimeWindow window, Iterable values, Collector out) throws Exception { + for(Integer val : values) { + out.collect(val); + } + } + }); + + outputStreamOperator.getSideOutput(new LateArrivingOutputTag()) + .flatMap(new FlatMapFunction, String>() { + @Override + public void flatMap(StreamRecord value, Collector out) throws Exception { + out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); } - } - }).print(); - - allWindowedStream.tooLateEvents().flatMap(new FlatMapFunction, String>() { - @Override - public void flatMap(StreamRecord value, Collector out) throws Exception { - out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); - } - }).print(); - + }).print(); see.execute(); } @@ -179,25 +178,29 @@ public void testKeyedWindowLateArrivingEvents() throws Exception { see.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); DataStream dataStream = see.fromCollection(elements); - WindowedStream windowStream = dataStream.assignTimestampsAndWatermarks( - new TestWatermarkAssigner()).keyBy(new TestKeySelector()).timeWindow(Time.milliseconds(1), Time.milliseconds(1)); - windowStream.apply(new WindowFunction() { + SingleOutputStreamOperator outputStreamOperator = dataStream.assignTimestampsAndWatermarks( + new TestWatermarkAssigner()).keyBy(new TestKeySelector()).timeWindow(Time.milliseconds(1), Time.milliseconds(1)) + .apply(new WindowFunction() { @Override public void apply(Integer key, TimeWindow window, Iterable input, Collector out) throws Exception { + CollectorWrapper sideOuput = new CollectorWrapper(out); for(Integer val : input) { out.collect(String.valueOf(key) + "-"+String.valueOf(val)); + sideOuput.collect(new SideOutputTag("applySideOutput"), "apply-" + String.valueOf(val)); } } - }).print(); + }); - windowStream.tooLateEvents().flatMap(new FlatMapFunction, String>() { - @Override - public void flatMap(StreamRecord value, Collector out) throws Exception { - out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); - } - }).print(); + outputStreamOperator.getSideOutput(new SideOutputTag("applySideOutput")).print(); + outputStreamOperator.getSideOutput(new LateArrivingOutputTag()) + .flatMap(new FlatMapFunction, String>() { + @Override + public void flatMap(StreamRecord value, Collector out) throws Exception { + out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); + } + }).print(); see.execute(); } From 62a26dc7cdd7cdb6474b4d8059f8d3a62909bb50 Mon Sep 17 00:00:00 2001 From: Chen Qin Date: Wed, 8 Feb 2017 22:03:25 -0800 Subject: [PATCH 4/5] merge with master --- .../api/datastream/WindowedStream.java | 1 + .../ReduceIterableAllWindowFunction.java | 10 +++++ .../streaming/api/graph/StreamGraph.java | 18 ++++----- .../api/graph/StreamingJobGraphGenerator.java | 1 - .../windowing/WindowOperatorTest.java | 40 ++++++------------- .../streaming/api/scala/WindowedStream.scala | 4 +- 6 files changed, 34 insertions(+), 40 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java index e42726670ab5c..93499f214d87e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java @@ -39,6 +39,7 @@ import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator; import org.apache.flink.streaming.api.functions.aggregation.SumAggregator; import org.apache.flink.streaming.api.functions.windowing.AggregateApplyWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.DiscardWindowFunction; import org.apache.flink.streaming.api.functions.windowing.FoldApplyWindowFunction; import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ReduceApplyWindowFunction; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java index 8ec5809ac93c5..1bfcb10ad8f5c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.streaming.api.windowing.windows.Window; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; @Internal @@ -45,4 +46,13 @@ public void apply(W window, Iterable input, Collector out) throws Exceptio } out.collect(curr); } + + /** + * discard non sideoutputs of keyed window, pass through sideOutputs only + */ + @Internal + public static class DiscardWindowFunction implements WindowFunction{ + @Override + public void apply(K k, W window, Iterable input, Collector out) throws Exception {} + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index e9eae3bcd4be3..b015d112dfe8a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -110,7 +110,7 @@ public void clear() { streamNodes = new HashMap<>(); virtualSelectNodes = new HashMap<>(); virtualOutputNodes = new HashMap<>(); - virtualPartitionNodes = new HashMap<>(); + virtuaPartitionNodes = new HashMap<>(); vertexIDtoBrokerID = new HashMap<>(); vertexIDtoLoopTimeout = new HashMap<>(); iterationSourceSinkPairs = new HashSet<>(); @@ -321,11 +321,11 @@ public void addVirtualOutputNode(Integer originalId, Integer virtualId, OutputTa */ public void addVirtualPartitionNode(Integer originalId, Integer virtualId, StreamPartitioner partitioner) { - if (virtualPartitionNodes.containsKey(virtualId)) { + if (virtuaPartitionNodes.containsKey(virtualId)) { throw new IllegalStateException("Already has virtual partition node with id " + virtualId); } - virtualPartitionNodes.put(virtualId, + virtuaPartitionNodes.put(virtualId, new Tuple2>(originalId, partitioner)); } @@ -339,8 +339,8 @@ public String getSlotSharingGroup(Integer id) { } else if (virtualSelectNodes.containsKey(id)) { Integer mappedId = virtualSelectNodes.get(id).f0; return getSlotSharingGroup(mappedId); - } else if (virtualPartitionNodes.containsKey(id)) { - Integer mappedId = virtualPartitionNodes.get(id).f0; + } else if (virtuaPartitionNodes.containsKey(id)) { + Integer mappedId = virtuaPartitionNodes.get(id).f0; return getSlotSharingGroup(mappedId); } else { StreamNode node = getStreamNode(id); @@ -383,9 +383,9 @@ private void addEdgeInternal(Integer upStreamVertexID, addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames, null); } else if (virtuaPartitionNodes.containsKey(upStreamVertexID)) { int virtualId = upStreamVertexID; - upStreamVertexID = virtualPartitionNodes.get(virtualId).f0; + upStreamVertexID = virtuaPartitionNodes.get(virtualId).f0; if (partitioner == null) { - partitioner = virtualPartitionNodes.get(virtualId).f1; + partitioner = virtuaPartitionNodes.get(virtualId).f1; } addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames, outputTag); @@ -418,8 +418,8 @@ private void addEdgeInternal(Integer upStreamVertexID, } public void addOutputSelector(Integer vertexID, OutputSelector outputSelector) { - if (virtualPartitionNodes.containsKey(vertexID)) { - addOutputSelector(virtualPartitionNodes.get(vertexID).f0, outputSelector); + if (virtuaPartitionNodes.containsKey(vertexID)) { + addOutputSelector(virtuaPartitionNodes.get(vertexID).f0, outputSelector); } else if (virtualSelectNodes.containsKey(vertexID)) { addOutputSelector(virtualSelectNodes.get(vertexID).f0, outputSelector); } else { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 389ba69bcb580..f1a8313077c1c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -53,7 +53,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index 762c7d2fb0ed7..6caf7bb65cec9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -66,7 +66,6 @@ import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestHarnessUtil; -import org.apache.flink.streaming.util.WindowingTestHarness; import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -1470,12 +1469,6 @@ public void testSideOutputDueToLatenessSessionZeroLatenessPurgingTrigger() throw @Test public void testSideOutputDueToLatenessSessionZeroLateness() throws Exception { - // same as testSideOutputDueToLatenessSessionZeroLateness() but with an accumulating trigger, i.e. - // one that does not return FIRE_AND_PURGE when firing but just FIRE - - // this has the same output as testDropDueToLatenessSessionZeroLateness() because - // accumulating/discarding does not make a difference with "allowed lateness" = 0. - final int GAP_SIZE = 3; final long LATENESS = 0; @@ -1559,9 +1552,9 @@ public void testSideOutputDueToLatenessSessionZeroLateness() throws Exception { } @Test - public void testSideOutputDueToLatenessSessionWithLatenessPurgingTrigger() throws Exception { + public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exception { - // this has the same output as testDropDueToLatenessSessionZeroLateness() because + // this has the same output as testSideOutputDueToLatenessSessionZeroLateness() because // the allowed lateness is too small to make a difference final int GAP_SIZE = 3; @@ -1590,8 +1583,7 @@ public void testSideOutputDueToLatenessSessionWithLatenessPurgingTrigger() throw testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); - ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); - + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1627,8 +1619,6 @@ public void testSideOutputDueToLatenessSessionWithLatenessPurgingTrigger() throw expected.add(new Watermark(14600)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000)); - // sideoutput as late - sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10000, tag)); expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 10000L, 14600L), 14599)); @@ -1643,15 +1633,12 @@ public void testSideOutputDueToLatenessSessionWithLatenessPurgingTrigger() throw ConcurrentLinkedQueue actual = testHarness.getOutput(); - ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); - TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); - testHarness.close(); } @Test - public void testSideOutputDueToLatenessSessionWithLateness() throws Exception { + public void testNotSideOutputDueToLatenessSessionWithLateness() throws Exception { // same as testSideOutputDueToLatenessSessionWithLateness() but with an accumulating trigger, i.e. // one that does not return FIRE_AND_PURGE when firing but just FIRE. The expected // results are therefore slightly different. @@ -1682,7 +1669,6 @@ public void testSideOutputDueToLatenessSessionWithLateness() throws Exception { testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); - ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1734,7 +1720,7 @@ public void testSideOutputDueToLatenessSessionWithLateness() throws Exception { ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); - TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", new ConcurrentLinkedQueue<>(), sideActual, new Tuple3ResultSortComparator()); testHarness.processWatermark(new Watermark(20000)); @@ -1748,13 +1734,13 @@ public void testSideOutputDueToLatenessSessionWithLateness() throws Exception { actual = testHarness.getOutput(); sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); - TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", new ConcurrentLinkedQueue<>(), sideActual, new Tuple3ResultSortComparator()); testHarness.close(); } @Test - public void testSideOutputDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws Exception { + public void testNotSideOutputDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws Exception { final int GAP_SIZE = 3; final long LATENESS = 10000; @@ -1782,7 +1768,6 @@ public void testSideOutputDueToLatenessSessionWithHugeLatenessPurgingTrigger() t testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); - ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1825,7 +1810,7 @@ public void testSideOutputDueToLatenessSessionWithHugeLatenessPurgingTrigger() t ConcurrentLinkedQueue actual = testHarness.getOutput(); ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); - TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", new ConcurrentLinkedQueue<>(), sideActual, new Tuple2ResultSortComparator()); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1840,12 +1825,12 @@ public void testSideOutputDueToLatenessSessionWithHugeLatenessPurgingTrigger() t actual = testHarness.getOutput(); sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); - TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple2ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", new ConcurrentLinkedQueue<>(), sideActual, new Tuple2ResultSortComparator()); testHarness.close(); } @Test - public void testSideOutputDueToLatenessSessionWithHugeLateness() throws Exception { + public void testNotSideOutputDueToLatenessSessionWithHugeLateness() throws Exception { final int GAP_SIZE = 3; final long LATENESS = 10000; @@ -1872,7 +1857,6 @@ public void testSideOutputDueToLatenessSessionWithHugeLateness() throws Exceptio testHarness.open(); ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>(); - ConcurrentLinkedQueue sideExpected = new ConcurrentLinkedQueue<>(); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); testHarness.processWatermark(new Watermark(1999)); @@ -1917,7 +1901,7 @@ public void testSideOutputDueToLatenessSessionWithHugeLateness() throws Exceptio ConcurrentLinkedQueue actual = testHarness.getOutput(); ConcurrentLinkedQueue sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); - TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", new ConcurrentLinkedQueue<>(), sideActual, new Tuple3ResultSortComparator()); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1932,7 +1916,7 @@ public void testSideOutputDueToLatenessSessionWithHugeLateness() throws Exceptio sideActual = testHarness.getSideOutput(tag); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator()); - TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", sideExpected, sideActual, new Tuple3ResultSortComparator()); + TestHarnessUtil.assertOutputEqualsSorted("SideOutput was not correct.", new ConcurrentLinkedQueue<>(), sideActual, new Tuple3ResultSortComparator()); testHarness.close(); } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala index e7eb7626ab0d7..e7a9276bd0bb5 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala @@ -18,13 +18,13 @@ package org.apache.flink.streaming.api.scala -import org.apache.flink.annotation.{PublicEvolving, Public} +import org.apache.flink.annotation.{Public, PublicEvolving} import org.apache.flink.api.common.functions.{AggregateFunction, FoldFunction, ReduceFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} -import org.apache.flink.streaming.api.functions.windowing.DiscardWindowFunction +import org.apache.flink.streaming.api.functions.windowing.ReduceIterableAllWindowFunction.DiscardWindowFunction import org.apache.flink.streaming.api.scala.function.WindowFunction import org.apache.flink.streaming.api.scala.function.util.{ScalaFoldFunction, ScalaReduceFunction, ScalaWindowFunction, ScalaWindowFunctionWrapper} import org.apache.flink.streaming.api.windowing.evictors.Evictor From e35bc35a32c0d032819cde8dadf36ed5adf9b799 Mon Sep 17 00:00:00 2001 From: Chen Qin Date: Fri, 10 Feb 2017 22:21:42 -0800 Subject: [PATCH 5/5] include outputtag as second parameters as window.apply(func, tag) --- .../flink/api/common/typeinfo/OutputTag.java | 1 - .../examples/wordcount/WordCount.java | 16 +-- .../api/datastream/AllWindowedStream.java | 33 ++++--- .../streaming/api/datastream/DataStream.java | 98 +++++++++---------- .../api/datastream/WindowedStream.java | 31 +++--- .../windowing/DiscardAllWindowFunction.java | 32 ------ .../windowing/DiscardWindowFunction.java | 32 ------ .../ReduceIterableAllWindowFunction.java | 10 -- .../streaming/api/graph/StreamGraph.java | 2 +- .../api/operators/TimestampedCollector.java | 2 +- .../operators/windowing/WindowOperator.java | 29 +++++- .../runtime/streamrecord/StreamRecord.java | 8 +- .../outputtags/LateArrivingOutputTag.java | 34 ------- .../windowing/WindowOperatorTest.java | 29 +++--- .../windowing/WindowTranslationTest.java | 36 ++++--- .../api/scala/AllWindowedStream.scala | 33 ++++--- .../streaming/api/scala/WindowedStream.scala | 34 ++++--- .../streaming/runtime/SideOutputITCase.java | 75 ++++++++------ 18 files changed, 247 insertions(+), 288 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java index 6ee4d15be3b73..ad394aa9221bc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/OutputTag.java @@ -57,7 +57,6 @@ public T getValue() { return value; } - @Override public boolean equals(Object obj) { return obj instanceof OutputTag diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java index 5ec1b22ec47f4..4290878dab02d 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java @@ -28,15 +28,15 @@ /** * Implements the "WordCount" program that computes a simple word occurrence * histogram over text files in a streaming fashion. - * + * *

    * The input is a plain text file with lines separated by newline characters. - * + * *

    * Usage: WordCount --input <path> --output <path>
    * If no parameters are provided, the program is run with default data from * {@link WordCountData}. - * + * *

    * This example shows how to: *

      @@ -44,7 +44,7 @@ *
    • use tuple data types, *
    • write and use user-defined functions. *
    - * + * */ public class WordCount { @@ -76,9 +76,9 @@ public static void main(String[] args) throws Exception { } DataStream> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) - text.flatMap(new Tokenizer()) - // group by the tuple field "0" and sum up tuple field "1" + // split up the lines in pairs (2-tuples) containing: (word,1) + text.flatMap(new Tokenizer()) + // group by the tuple field "0" and sum up tuple field "1" .keyBy(0).sum(1); // emit result @@ -108,7 +108,7 @@ public static final class Tokenizer implements FlatMapFunction> out) - throws Exception { + throws Exception { // normalize and split the line String[] tokens = value.toLowerCase().split("\\W+"); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java index b4c62f0b4430f..dc55e9556b9a3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.state.FoldingStateDescriptor; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingStateDescriptor; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.Utils; @@ -39,7 +40,6 @@ import org.apache.flink.streaming.api.functions.aggregation.SumAggregator; import org.apache.flink.streaming.api.functions.windowing.AggregateApplyAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; -import org.apache.flink.streaming.api.functions.windowing.DiscardAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.PassThroughAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.FoldApplyAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ReduceApplyAllWindowFunction; @@ -56,7 +56,6 @@ import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueAllWindowFunction; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -647,31 +646,32 @@ public SingleOutputStreamOperator apply(AllWindowFunction functi TypeInformation resultType = TypeExtractor.getUnaryOperatorReturnType( function, AllWindowFunction.class, true, true, getInputType(), null, false); - return apply(function, resultType); + return applyInternal(function, resultType, null); } /** - * Applies a DiscardWindowFunction that only returns late arriving events - * @return the data stream considered too late to be evaluated by any windows assigned + * Same as apply above except all window function emits late arriving input events with assigned OutputTag + * @param function The window function. + * @param tag OutputTag of skipped late arriving events + * @return The data stream that is the result of applying the window function to the window. */ - public DataStream> tooLateEvents() { - return apply(new DiscardAllWindowFunction()).getSideOutput(new LateArrivingOutputTag()); + public SingleOutputStreamOperator apply(AllWindowFunction function, OutputTag tag) { + TypeInformation resultType = TypeExtractor.getUnaryOperatorReturnType( + function, AllWindowFunction.class, true, true, getInputType(), null, false); + + return applyInternal(function, resultType, tag); } /** - * Applies the given window function to each window. The window function is called for each - * evaluation of the window for each key individually. The output of the window function is - * interpreted as a regular non-windowed stream. - * - *

    - * Not that this function requires that all data in the windows is buffered until the window - * is evaluated, as the function provides no means of incremental aggregation. - * + * Maintain binary compatible * @param function The window function. - * @param resultType Type information for the result type of the window function * @return The data stream that is the result of applying the window function to the window. */ public SingleOutputStreamOperator apply(AllWindowFunction function, TypeInformation resultType) { + return applyInternal(function, resultType, null); + } + + private SingleOutputStreamOperator applyInternal(AllWindowFunction function, TypeInformation resultType, OutputTag tag) { //clean the closure function = input.getExecutionEnvironment().clean(function); @@ -722,6 +722,7 @@ public SingleOutputStreamOperator apply(AllWindowFunction functi allowedLateness); } + operator.setLateArrivingTag(tag); return input.transform(opName, resultType, operator).forceNonParallel(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 7f25bb249ecbb..204557db0100a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -192,9 +192,9 @@ public final DataStream union(DataStream... streams) { for (DataStream newStream : streams) { if (!getType().equals(newStream.getType())) { throw new IllegalArgumentException("Cannot union streams of different types: " - + getType() + " and " + newStream.getType()); + + getType() + " and " + newStream.getType()); } - + unionedTransforms.add(newStream.getTransformation()); } return new DataStream<>(this.environment, new UnionTransformation<>(unionedTransforms)); @@ -232,7 +232,7 @@ public ConnectedStreams connect(DataStream dataStream) { /** * * It creates a new {@link KeyedStream} that uses the provided key for partitioning - * its operator states. + * its operator states. * * @param key * The KeySelector to be used for extracting the key for partitioning @@ -243,7 +243,7 @@ public KeyedStream keyBy(KeySelector key) { } /** - * Partitions the operator state of a {@link DataStream} by the given key positions. + * Partitions the operator state of a {@link DataStream} by the given key positions. * * @param fields * The position of the fields on which the {@link DataStream} @@ -259,7 +259,7 @@ public KeyedStream keyBy(int... fields) { } /** - * Partitions the operator state of a {@link DataStream}using field expressions. + * Partitions the operator state of a {@link DataStream}using field expressions. * A field expression is either the name of a public field or a getter method with parentheses * of the {@link DataStream}S underlying type. A dot can be used to drill * down into objects, as in {@code "field1.getInnerField2()" }. @@ -275,7 +275,7 @@ public KeyedStream keyBy(String... fields) { private KeyedStream keyBy(Keys keys) { return new KeyedStream<>(this, clean(KeySelectorUtil.getSelectorForKeys(keys, - getType(), getExecutionConfig()))); + getType(), getExecutionConfig()))); } /** @@ -326,7 +326,7 @@ public DataStream partitionCustom(Partitioner partitioner, String fiel */ public DataStream partitionCustom(Partitioner partitioner, KeySelector keySelector) { return setConnectionType(new CustomPartitionerWrapper<>(clean(partitioner), - clean(keySelector))); + clean(keySelector))); } // private helper method for custom partitioning @@ -334,9 +334,9 @@ private DataStream partitionCustom(Partitioner partitioner, Keys ke KeySelector keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig()); return setConnectionType( - new CustomPartitionerWrapper<>( - clean(partitioner), - clean(keySelector))); + new CustomPartitionerWrapper<>( + clean(partitioner), + clean(keySelector))); } /** @@ -504,7 +504,7 @@ public IterativeStream iterate(long maxWaitTimeMillis) { public SingleOutputStreamOperator map(MapFunction mapper) { TypeInformation outType = TypeExtractor.getMapReturnTypes(clean(mapper), getType(), - Utils.getCallLocationName(), true); + Utils.getCallLocationName(), true); return transform("Map", outType, new StreamMap<>(clean(mapper))); } @@ -528,7 +528,7 @@ public SingleOutputStreamOperator map(MapFunction mapper) { public SingleOutputStreamOperator flatMap(FlatMapFunction flatMapper) { TypeInformation outType = TypeExtractor.getFlatMapReturnTypes(clean(flatMapper), - getType(), Utils.getCallLocationName(), true); + getType(), Utils.getCallLocationName(), true); return transform("Flat Map", outType, new StreamFlatMap<>(clean(flatMapper))); @@ -667,8 +667,8 @@ public AllWindowedStream countWindowAll(long size) { */ public AllWindowedStream countWindowAll(long size, long slide) { return windowAll(GlobalWindows.create()) - .evictor(CountEvictor.of(size)) - .trigger(CountTrigger.of(slide)); + .evictor(CountEvictor.of(size)) + .trigger(CountTrigger.of(slide)); } /** @@ -698,7 +698,7 @@ public AllWindowedStream windowAll(WindowAssigner AllWindowedStream windowAll(WindowAssigner assignTimestamps(TimestampExtractor extr int inputParallelism = getTransformation().getParallelism(); ExtractTimestampsOperator operator = new ExtractTimestampsOperator<>(clean(extractor)); return transform("ExtractTimestamps", getTransformation().getOutputType(), operator) - .setParallelism(inputParallelism); + .setParallelism(inputParallelism); } /** * Assigns timestamps to the elements in the data stream and periodically creates * watermarks to signal event time progress. - * + * *

    This method creates watermarks periodically (for example every second), based * on the watermarks indicated by the given watermark generator. Even when no new elements * in the stream arrive, the given watermark generator will be periodically checked for * new watermarks. The interval in which watermarks are generated is defined in * {@link ExecutionConfig#setAutoWatermarkInterval(long)}. - * + * *

    Use this method for the common cases, where some characteristic over all elements * should generate the watermarks, or where watermarks are simply trailing behind the * wall clock time by a certain amount. @@ -746,33 +746,33 @@ public SingleOutputStreamOperator assignTimestamps(TimestampExtractor extr * timestamp seen so far in the elements of the stream by a fixed amount of time, and this * amount is known in advance, use the * {@link BoundedOutOfOrdernessTimestampExtractor}. - * + * *

    For cases where watermarks should be created in an irregular fashion, for example * based on certain markers that some element carry, use the * {@link AssignerWithPunctuatedWatermarks}. - * + * * @param timestampAndWatermarkAssigner The implementation of the timestamp assigner and - * watermark generator. + * watermark generator. * @return The stream after the transformation, with assigned timestamps and watermarks. - * + * * @see AssignerWithPeriodicWatermarks * @see AssignerWithPunctuatedWatermarks - * @see #assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks) + * @see #assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks) */ public SingleOutputStreamOperator assignTimestampsAndWatermarks( - AssignerWithPeriodicWatermarks timestampAndWatermarkAssigner) { - + AssignerWithPeriodicWatermarks timestampAndWatermarkAssigner) { + // match parallelism to input, otherwise dop=1 sources could lead to some strange // behaviour: the watermark will creep along very slowly because the elements // from the source go to each extraction operator round robin. final int inputParallelism = getTransformation().getParallelism(); final AssignerWithPeriodicWatermarks cleanedAssigner = clean(timestampAndWatermarkAssigner); - - TimestampsAndPeriodicWatermarksOperator operator = - new TimestampsAndPeriodicWatermarksOperator<>(cleanedAssigner); - + + TimestampsAndPeriodicWatermarksOperator operator = + new TimestampsAndPeriodicWatermarksOperator<>(cleanedAssigner); + return transform("Timestamps/Watermarks", getTransformation().getOutputType(), operator) - .setParallelism(inputParallelism); + .setParallelism(inputParallelism); } /** @@ -786,7 +786,7 @@ public SingleOutputStreamOperator assignTimestampsAndWatermarks( * non-negative and greater than the previous watermark. * *

    This method is useful when the data stream embeds watermark elements, or certain elements - * carry a marker that can be used to determine the current event time watermark. + * carry a marker that can be used to determine the current event time watermark. * This operation gives the programmer full control over the watermark generation. Users * should be aware that too aggressive watermark generation (i.e., generating hundreds of * watermarks every second) can cost some performance. @@ -795,7 +795,7 @@ public SingleOutputStreamOperator assignTimestampsAndWatermarks( * every x milliseconds, use the {@link AssignerWithPeriodicWatermarks}. * * @param timestampAndWatermarkAssigner The implementation of the timestamp assigner and - * watermark generator. + * watermark generator. * @return The stream after the transformation, with assigned timestamps and watermarks. * * @see AssignerWithPunctuatedWatermarks @@ -803,25 +803,25 @@ public SingleOutputStreamOperator assignTimestampsAndWatermarks( * @see #assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks) */ public SingleOutputStreamOperator assignTimestampsAndWatermarks( - AssignerWithPunctuatedWatermarks timestampAndWatermarkAssigner) { - + AssignerWithPunctuatedWatermarks timestampAndWatermarkAssigner) { + // match parallelism to input, otherwise dop=1 sources could lead to some strange // behaviour: the watermark will creep along very slowly because the elements // from the source go to each extraction operator round robin. final int inputParallelism = getTransformation().getParallelism(); final AssignerWithPunctuatedWatermarks cleanedAssigner = clean(timestampAndWatermarkAssigner); - TimestampsAndPunctuatedWatermarksOperator operator = - new TimestampsAndPunctuatedWatermarksOperator<>(cleanedAssigner); - + TimestampsAndPunctuatedWatermarksOperator operator = + new TimestampsAndPunctuatedWatermarksOperator<>(cleanedAssigner); + return transform("Timestamps/Watermarks", getTransformation().getOutputType(), operator) - .setParallelism(inputParallelism); + .setParallelism(inputParallelism); } // ------------------------------------------------------------------------ // Data sinks // ------------------------------------------------------------------------ - + /** * Writes a DataStream to the standard output stream (stdout). * @@ -954,10 +954,10 @@ public DataStreamSink writeAsCsv(String path, WriteMode writeMode) { @SuppressWarnings("unchecked") @PublicEvolving public DataStreamSink writeAsCsv( - String path, - WriteMode writeMode, - String rowDelimiter, - String fieldDelimiter) { + String path, + WriteMode writeMode, + String rowDelimiter, + String fieldDelimiter) { Preconditions.checkArgument( getType().isTupleType(), "The writeAsCsv() method can only be used on data streams of tuples."); @@ -1029,11 +1029,11 @@ public SingleOutputStreamOperator transform(String operatorName, TypeInfo transformation.getOutputType(); OneInputTransformation resultTransform = new OneInputTransformation<>( - this.transformation, - operatorName, - operator, - outTypeInfo, - environment.getParallelism()); + this.transformation, + operatorName, + operator, + outTypeInfo, + environment.getParallelism()); @SuppressWarnings({ "unchecked", "rawtypes" }) SingleOutputStreamOperator returnStream = new SingleOutputStreamOperator(environment, resultTransform); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java index 93499f214d87e..6cc75ba169a0a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.state.FoldingStateDescriptor; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingStateDescriptor; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.Utils; @@ -39,7 +40,6 @@ import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator; import org.apache.flink.streaming.api.functions.aggregation.SumAggregator; import org.apache.flink.streaming.api.functions.windowing.AggregateApplyWindowFunction; -import org.apache.flink.streaming.api.functions.windowing.DiscardWindowFunction; import org.apache.flink.streaming.api.functions.windowing.FoldApplyWindowFunction; import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ReduceApplyWindowFunction; @@ -66,7 +66,6 @@ import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -721,6 +720,18 @@ public SingleOutputStreamOperator apply(WindowFunction functi return apply(function, resultType); } + /** + * Same as apply above except window function emits late arriving input events with assigned OutputTag + * @param function The window function. + * @param tag OutputTag + * @return The data stream that is the result of applying the window function to the window. + */ + public SingleOutputStreamOperator apply(WindowFunction function, OutputTag tag) { + TypeInformation resultType = TypeExtractor.getUnaryOperatorReturnType( + function, WindowFunction.class, true, true, getInputType(), null, false); + return applyInternal(function, resultType, tag); + } + /** * Applies the given window function to each window. The window function is called for each * evaluation of the window for each key individually. The output of the window function is @@ -734,7 +745,12 @@ public SingleOutputStreamOperator apply(WindowFunction functi * @param resultType Type information for the result type of the window function * @return The data stream that is the result of applying the window function to the window. */ - public SingleOutputStreamOperator apply(WindowFunction function, TypeInformation resultType) { + + public SingleOutputStreamOperator apply(WindowFunction function, TypeInformation resultType){ + return applyInternal(function, resultType, null); + } + + private SingleOutputStreamOperator applyInternal(WindowFunction function, TypeInformation resultType, OutputTag tag) { //clean the closure function = input.getExecutionEnvironment().clean(function); @@ -792,6 +808,7 @@ public SingleOutputStreamOperator apply(WindowFunction functi legacyWindowOpType); } + operator.setLateArrivingTag(tag); return input.transform(opName, resultType, operator); } @@ -993,14 +1010,6 @@ public SingleOutputStreamOperator apply(R initialValue, FoldFunction> tooLateEvents() { - return apply(new DiscardWindowFunction()).getSideOutput(new LateArrivingOutputTag()); - } - // ------------------------------------------------------------------------ // Pre-defined aggregations on the keyed windows // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java deleted file mode 100644 index aafc0e7291b8e..0000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardAllWindowFunction.java +++ /dev/null @@ -1,32 +0,0 @@ -/** - * 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.windowing; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.streaming.api.windowing.windows.Window; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.util.Collector; - -/** - * discard non sideoutputs - */ -@Internal -public class DiscardAllWindowFunction implements AllWindowFunction{ - @Override - public void apply(W window, Iterable values, Collector out) throws Exception {} -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java deleted file mode 100644 index 5ce3b3d547f13..0000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/DiscardWindowFunction.java +++ /dev/null @@ -1,32 +0,0 @@ -/** - * 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.windowing; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.streaming.api.windowing.windows.Window; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.util.Collector; - -/** - * discard non sideoutputs of keyed window, pass through sideOutputs only - */ -@Internal -public class DiscardWindowFunction implements WindowFunction{ - @Override - public void apply(K k, W window, Iterable input, Collector out) throws Exception {} -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java index 1bfcb10ad8f5c..8ec5809ac93c5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceIterableAllWindowFunction.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.streaming.api.windowing.windows.Window; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; @Internal @@ -46,13 +45,4 @@ public void apply(W window, Iterable input, Collector out) throws Exceptio } out.collect(curr); } - - /** - * discard non sideoutputs of keyed window, pass through sideOutputs only - */ - @Internal - public static class DiscardWindowFunction implements WindowFunction{ - @Override - public void apply(K k, W window, Iterable input, Collector out) throws Exception {} - } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index b015d112dfe8a..90f14012f6266 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -380,7 +380,7 @@ private void addEdgeInternal(Integer upStreamVertexID, outputNames = virtualSelectNodes.get(virtualId).f1; } - addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames, null); + addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames, outputTag); } else if (virtuaPartitionNodes.containsKey(upStreamVertexID)) { int virtualId = upStreamVertexID; upStreamVertexID = virtuaPartitionNodes.get(virtualId).f0; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java index 5113f4b4112ea..8976b7df36e3b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java @@ -31,7 +31,7 @@ * would set the timestamp of the incoming * {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord} here. * - * @param The type of the elements that can be emitted. + * @param The type of the elements it emits for non sideoutput. */ @Internal public class TimestampedCollector implements RichCollector { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java index eab64f7d5544b..45213f5b02be0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; @@ -63,7 +64,6 @@ import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import org.apache.flink.util.Preconditions; import java.io.IOException; @@ -133,6 +133,12 @@ public class WindowOperator */ private final long allowedLateness; + /** + * OutputTag used in late arriving events. This is used for + * sideoutputs events with timestamp pass {@code window.maxTimestamp + allowedLateness} + */ + private OutputTag lateArrivingTag = null; + // ------------------------------------------------------------------------ // State that is not checkpointed // ------------------------------------------------------------------------ @@ -245,6 +251,15 @@ public WindowOperator( setChainingStrategy(ChainingStrategy.ALWAYS); } + /** + * user defined OutputTag to collect late arriving events + * by default is null + * @param lateArrivingTag OutputTag to mark late arriving events processed by a window operator + */ + public void setLateArrivingTag(final OutputTag lateArrivingTag){ + this.lateArrivingTag = lateArrivingTag; + } + @Override public void open() throws Exception { super.open(); @@ -427,7 +442,11 @@ public void merge(W mergeResult, } } - if(isSkippedElement && isLate(element)) { + // side output input event if + // element not handled by any window + // late arriving tag has been set + // windowAssigner is event time and current timestamp + allowed lateness no less than element timestamp + if(isSkippedElement && lateArrivingTag != null && isLate(element)) { sideOutput(element); } } @@ -558,10 +577,10 @@ private void emitWindowContents(W window, ACC contents) throws Exception { /** * write skipped late arriving element to SideOutput - * @param element element to side output + * @param element skipped late arriving element to side output */ private void sideOutput(StreamRecord element){ - timestampedCollector.collect(new LateArrivingOutputTag(), element); + timestampedCollector.collect(lateArrivingTag, element.getValue()); } /** @@ -588,7 +607,7 @@ protected boolean isLate(W window) { /** * Decide if a record is currently late, based on current watermark and allowed lateness * @param element The element to check - * @return The element for which should be considered when sideoutput + * @return The element for which should be considered when sideoutputs */ protected boolean isLate(StreamRecord element){ return (windowAssigner.isEventTime()) && diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java index 474d148693201..ffaceac40c147 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.OutputTag; -import org.apache.flink.api.common.typeinfo.TypeInformation; /** * One value in a data stream. This stores the value and an optional associated timestamp. @@ -47,15 +46,16 @@ public final class StreamRecord extends StreamElement { */ public StreamRecord(T value) { this.value = value; + this.tag = null; } - + /** + * Creates a new sideoutput StreamRecord. The record does not have a timestamp. + */ public StreamRecord(T value, OutputTag tag) { this.value = value; this.tag = requireNonNull(tag); } - public TypeInformation getType() { return tag == null ? null : tag.getTypeInfo();} - public OutputTag getOutputTag() { return tag; } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java deleted file mode 100644 index 22b475cd3b736..0000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/outputtags/LateArrivingOutputTag.java +++ /dev/null @@ -1,34 +0,0 @@ -/** - * 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.util.outputtags; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.OutputTag; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; - -/** - * {@code OutputTag} for late arriving events - * @param {@code WindowOperator} input type - */ -@Internal -public final class LateArrivingOutputTag extends OutputTag>{ - - public LateArrivingOutputTag(){ - super(); - } -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index 6caf7bb65cec9..35cc64714990c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; @@ -66,7 +67,6 @@ import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestHarnessUtil; -import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -91,7 +91,7 @@ public class WindowOperatorTest extends TestLogger { private static AtomicInteger closeCalled = new AtomicInteger(0); //late arriving event OutputTag> - private static final LateArrivingOutputTag tag = new LateArrivingOutputTag>(); + private static final OutputTag> tag = new OutputTag>() {}; private void testSlidingEventTimeWindows(OneInputStreamOperatorTestHarness, Tuple2> testHarness) throws Exception { @@ -1128,6 +1128,8 @@ public void testLateness() throws Exception { PurgingTrigger.of(EventTimeTrigger.create()), LATENESS); + operator.setLateArrivingTag(tag); + OneInputStreamOperatorTestHarness, Tuple2> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); @@ -1159,7 +1161,7 @@ public void testLateness() throws Exception { testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998)); testHarness.processWatermark(new Watermark(7000)); - lateExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1998, tag)); + lateExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1999, tag)); expected.add(new Watermark(7000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator()); @@ -1257,6 +1259,8 @@ public void testSideOutputDueToLatenessTumbling() throws Exception { EventTimeTrigger.create(), LATENESS); + operator.setLateArrivingTag(tag); + OneInputStreamOperatorTestHarness, Tuple2> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); @@ -1278,9 +1282,9 @@ public void testSideOutputDueToLatenessTumbling() throws Exception { expected.add(new StreamRecord<>(new Tuple2<>("key2", 2), 1999)); expected.add(new Watermark(1999)); - // sideoutput as late + // sideoutput as late, will reuse previous timestamp since only input tuple is sideoutputed testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998)); - sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1998, tag)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1999, tag)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 2001)); testHarness.processWatermark(new Watermark(2999)); @@ -1320,6 +1324,7 @@ public void testSideOutputDueToLatenessSliding() throws Exception { EventTimeTrigger.create(), LATENESS); + operator.setLateArrivingTag(tag); OneInputStreamOperatorTestHarness, Tuple2> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); @@ -1365,7 +1370,7 @@ public void testSideOutputDueToLatenessSliding() throws Exception { // sideoutput element due to lateness testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 3001)); - sideExpected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 3001, tag)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 5999, tag)); testHarness.processWatermark(new Watermark(25000)); @@ -1397,6 +1402,7 @@ public void testSideOutputDueToLatenessSessionZeroLatenessPurgingTrigger() throw new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()), PurgingTrigger.of(EventTimeTrigger.create()), LATENESS); + operator.setLateArrivingTag(tag); OneInputStreamOperatorTestHarness, Tuple3> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); @@ -1442,11 +1448,11 @@ public void testSideOutputDueToLatenessSessionZeroLatenessPurgingTrigger() throw // this is sideoutput as late testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000)); - sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10000, tag)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 14599, tag)); // this is also sideoutput as late (we test that they are not accidentally merged) testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10100)); - sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10100, tag)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 14599, tag)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1488,7 +1494,7 @@ public void testSideOutputDueToLatenessSessionZeroLateness() throws Exception { new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()), EventTimeTrigger.create(), LATENESS); - + operator.setLateArrivingTag(tag); OneInputStreamOperatorTestHarness, Tuple3> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); @@ -1531,9 +1537,9 @@ public void testSideOutputDueToLatenessSessionZeroLateness() throws Exception { expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 11600L, 14600L), 14599)); expected.add(new Watermark(14600)); - // this is sideoutput as late + // this is sideoutput as late, reuse last timestamp testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000)); - sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 10000, tag)); + sideExpected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 14599, tag)); testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500)); testHarness.processWatermark(new Watermark(20000)); @@ -1577,6 +1583,7 @@ public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exce PurgingTrigger.of(EventTimeTrigger.create()), LATENESS); + operator.setLateArrivingTag(tag); OneInputStreamOperatorTestHarness, Tuple3> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java index f72a2f14f237e..e773c499fb1d7 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java @@ -29,16 +29,19 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.OutputTag; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.datastream.WindowedStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.windowing.WindowFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows; @@ -746,22 +749,29 @@ public void testApplyEventTime() throws Exception { DataStream> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); - DataStream> window1 = source + final OutputTag> lateTag = new OutputTag>() {}; + + SingleOutputStreamOperator> window1 = source .keyBy(new TupleKeySelector()) .window(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) - .apply(new WindowFunction, Tuple2, String, TimeWindow>() { - private static final long serialVersionUID = 1L; - - @Override - public void apply(String key, - TimeWindow window, - Iterable> values, - Collector> out) throws Exception { - for (Tuple2 in : values) { - out.collect(in); + .apply( + new WindowFunction, Tuple2, String, TimeWindow>() { + private static final long serialVersionUID = 1L; + + @Override + public void apply(String key, + TimeWindow window, + Iterable> values, + Collector> out) throws Exception { + for (Tuple2 in : values) { + out.collect(in); + } } - } - }); + }, + lateTag + ); + + window1.getSideOutput(lateTag); OneInputTransformation, Tuple2> transform = (OneInputTransformation, Tuple2>) window1.getTransformation(); OneInputStreamOperator, Tuple2> operator = transform.getOperator(); diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala index e201bd3a2264e..9c9fed326b1bf 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala @@ -20,21 +20,17 @@ package org.apache.flink.streaming.api.scala import org.apache.flink.annotation.{PublicEvolving, Public} import org.apache.flink.api.common.functions.{AggregateFunction, FoldFunction, ReduceFunction} -import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.common.typeinfo.{OutputTag, TypeInformation} import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWStream} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} -import org.apache.flink.streaming.api.functions.windowing.{DiscardAllWindowFunction} import org.apache.flink.streaming.api.scala.function.AllWindowFunction import org.apache.flink.streaming.api.scala.function.util.{ScalaAllWindowFunction, ScalaAllWindowFunctionWrapper, ScalaFoldFunction, ScalaReduceFunction} import org.apache.flink.streaming.api.windowing.evictors.Evictor import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.triggers.Trigger import org.apache.flink.streaming.api.windowing.windows.Window -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord -import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag import org.apache.flink.util.Collector - import org.apache.flink.util.Preconditions.checkNotNull /** @@ -415,6 +411,23 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) { asScalaStream(javaStream.apply(javaFunction, implicitly[TypeInformation[R]])) } + /** + * Same as apply above except all window function + * emits late arriving input events with assigned OutputTag + * + * @param function The window function. + * @param tag OutputTag + * @return The data stream that is the result of applying the window function to the window. + */ + def apply[R: TypeInformation]( + function: AllWindowFunction[T, R, W], tag: OutputTag[T]): DataStream[R] = { + + val cleanedFunction = clean(function) + val javaFunction = new ScalaAllWindowFunctionWrapper[T, R, W](cleanedFunction) + + asScalaStream(javaStream.apply(javaFunction, tag)) + } + /** * Applies the given window function to each window. The window function is called for each * evaluation of the window for each key individually. The output of the window function is @@ -562,16 +575,6 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) { asScalaStream(javaStream.apply(initialValue, folder, applyFunction, returnType)) } - /** - * Applies a DiscardWindowFunction that only returns late arriving events - * @return the data stream considered too late to be evaluated by any windows - */ - def tooLateEvents(): DataStream[StreamRecord[T]] = { - val discardAllWindowFunction = new DiscardAllWindowFunction[T, W]() - val lateArrivingOutputTag = new LateArrivingOutputTag[T]() - asScalaStream(javaStream.apply(discardAllWindowFunction).getSideOutput(lateArrivingOutputTag)) - } - // ------------------------------------------------------------------------ // Aggregations on the keyed windows // ------------------------------------------------------------------------ diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala index e7a9276bd0bb5..75468f0f2e9a3 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala @@ -18,21 +18,18 @@ package org.apache.flink.streaming.api.scala -import org.apache.flink.annotation.{Public, PublicEvolving} +import org.apache.flink.annotation.{PublicEvolving, Public} import org.apache.flink.api.common.functions.{AggregateFunction, FoldFunction, ReduceFunction} -import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.common.typeinfo.{OutputTag, TypeInformation} import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} -import org.apache.flink.streaming.api.functions.windowing.ReduceIterableAllWindowFunction.DiscardWindowFunction import org.apache.flink.streaming.api.scala.function.WindowFunction import org.apache.flink.streaming.api.scala.function.util.{ScalaFoldFunction, ScalaReduceFunction, ScalaWindowFunction, ScalaWindowFunctionWrapper} import org.apache.flink.streaming.api.windowing.evictors.Evictor import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.triggers.Trigger import org.apache.flink.streaming.api.windowing.windows.Window -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord -import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag import org.apache.flink.util.Collector /** @@ -403,6 +400,23 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) { asScalaStream(javaStream.apply(applyFunction, implicitly[TypeInformation[R]])) } + /** + * Same as apply above except window function + * emits late arriving input events with assigned OutputTag + * + * @param function The window function. + * @param tag OutputTag + * @return The data stream that is the result of applying the window function to the window. + */ + def apply[R: TypeInformation]( + function: WindowFunction[T, R, K, W], tag: OutputTag[T]): DataStream[R] = { + + val cleanFunction = clean(function) + val applyFunction = new ScalaWindowFunctionWrapper[T, R, K, W](cleanFunction) + asScalaStream(javaStream.apply(applyFunction, tag)) + } + + /** * Applies the given window function to each window. The window function is called for each * evaluation of the window for each key individually. The output of the window function is @@ -551,16 +565,6 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) { asScalaStream(javaStream.apply(initialValue, folder, applyFunction, resultType)) } - /** - * Applies a DiscardWindowFunction that only returns late arriving events - * @return the data stream considered too late to be evaluated by any windows - */ - def tooLateEvents(): DataStream[StreamRecord[T]] = { - val discardWindowFunction = new DiscardWindowFunction[T, K, W]() - val lateArrivingOutputTag = new LateArrivingOutputTag[T]() - asScalaStream(javaStream.apply(discardWindowFunction).getSideOutput(lateArrivingOutputTag)) - } - // ------------------------------------------------------------------------ // Aggregations on the keyed windows // ------------------------------------------------------------------------ diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java index df4c1e8e2d609..7ee6e71ef3abd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java @@ -30,17 +30,18 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; -import org.apache.flink.streaming.util.outputtags.LateArrivingOutputTag; +import org.apache.flink.test.streaming.runtime.util.TestListResultSink; import org.apache.flink.util.Collector; import org.apache.flink.util.CollectorWrapper; import org.junit.Test; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import static org.junit.Assert.assertEquals; /** * Integration test for streaming programs using sideOutputs * @@ -66,13 +67,20 @@ public SideOutputTag(String value){ } } + /** + * Serializable outputTag used in late arriving events + */ + static class LateArrivingTag extends OutputTag{} + /** * Test flatMap sideOutputs */ @Test public void testFlatMapSideOutputs() throws Exception { + TestListResultSink sideOutputResultSink = new TestListResultSink<>(); + TestListResultSink resultSink = new TestListResultSink<>(); + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); - see.disableOperatorChaining(); see.setParallelism(3); DataStream dataStream = see.fromCollection(elements); @@ -87,11 +95,12 @@ public void flatMap(Integer value, Collector out) throws Exception { } }); - passThroughtStream.getSideOutput(new SideOutputTag("side")).print(); - passThroughtStream.print(); - - passThroughtStream.getSideOutput(new SideOutputTag("notside")).print(); + passThroughtStream.getSideOutput(new SideOutputTag("side")).addSink(sideOutputResultSink); + passThroughtStream.addSink(resultSink); see.execute(); + + assertEquals(sideOutputResultSink.getSortedResult(), Arrays.asList("sideout-1", "sideout-2", "sideout-3", "sideout-4", "sideout-5")); + assertEquals(resultSink.getSortedResult(), Arrays.asList(1, 2, 3, 4, 5)); } /** @@ -99,8 +108,9 @@ public void flatMap(Integer value, Collector out) throws Exception { */ @Test public void testFlatMapSideOutputsWithWrongTag() throws Exception { + TestListResultSink sideOutputResultSink = new TestListResultSink<>(); + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); - see.disableOperatorChaining(); see.setParallelism(3); DataStream dataStream = see.fromCollection(elements); @@ -113,8 +123,9 @@ public void flatMap(Integer value, Collector out) throws Exception { CollectorWrapper wrapper = new CollectorWrapper<>(out); wrapper.collect(new SideOutputTag("side"), "sideout-" + String.valueOf(value)); } - }).getSideOutput(new SideOutputTag("notside")).print(); + }).getSideOutput(new SideOutputTag("notside")).addSink(sideOutputResultSink); see.execute(); + assertEquals(sideOutputResultSink.getSortedResult(), Arrays.asList()); } private static class TestWatermarkAssigner implements AssignerWithPunctuatedWatermarks{ @@ -142,12 +153,14 @@ public Integer getKey(Integer value) throws Exception { */ @Test public void testAllWindowLateArrivingEvents() throws Exception { + TestListResultSink sideOutputResultSink = new TestListResultSink<>(); + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); - see.disableOperatorChaining(); - see.setParallelism(3); + see.setParallelism(1); see.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); DataStream dataStream = see.fromCollection(elements); + LateArrivingTag tag = new LateArrivingTag(); SingleOutputStreamOperator outputStreamOperator = dataStream.assignTimestampsAndWatermarks( new TestWatermarkAssigner()).timeWindowAll(Time.milliseconds(1), Time.milliseconds(1)) @@ -158,50 +171,52 @@ public void apply(TimeWindow window, Iterable values, Collector()) - .flatMap(new FlatMapFunction, String>() { + outputStreamOperator.getSideOutput(tag) + .flatMap(new FlatMapFunction() { @Override - public void flatMap(StreamRecord value, Collector out) throws Exception { - out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); + public void flatMap(Integer value, Collector out) throws Exception { + out.collect("late-" + String.valueOf(value)); } - }).print(); + }).addSink(sideOutputResultSink); see.execute(); + assertEquals(sideOutputResultSink.getSortedResult(), Arrays.asList("late-3", "late-4")); + } @Test public void testKeyedWindowLateArrivingEvents() throws Exception { + TestListResultSink resultSink = new TestListResultSink<>(); + TestListResultSink lateResultSink = new TestListResultSink<>(); + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); - see.disableOperatorChaining(); see.setParallelism(3); see.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); DataStream dataStream = see.fromCollection(elements); SingleOutputStreamOperator outputStreamOperator = dataStream.assignTimestampsAndWatermarks( - new TestWatermarkAssigner()).keyBy(new TestKeySelector()).timeWindow(Time.milliseconds(1), Time.milliseconds(1)) + new TestWatermarkAssigner()).keyBy(new TestKeySelector()).timeWindow( + Time.milliseconds(1), Time.milliseconds(1)).allowedLateness(Time.milliseconds(2)) .apply(new WindowFunction() { @Override public void apply(Integer key, TimeWindow window, Iterable input, Collector out) throws Exception { CollectorWrapper sideOuput = new CollectorWrapper(out); for(Integer val : input) { out.collect(String.valueOf(key) + "-"+String.valueOf(val)); - sideOuput.collect(new SideOutputTag("applySideOutput"), "apply-" + String.valueOf(val)); + sideOuput.collect(new SideOutputTag("ontime"), "ontime-" + String.valueOf(val)); } } - }); + }, new LateArrivingTag()); - outputStreamOperator.getSideOutput(new SideOutputTag("applySideOutput")).print(); - - outputStreamOperator.getSideOutput(new LateArrivingOutputTag()) - .flatMap(new FlatMapFunction, String>() { - @Override - public void flatMap(StreamRecord value, Collector out) throws Exception { - out.collect("late-" + String.valueOf(value.getValue()) + "-ts" + String.valueOf(value.getTimestamp())); - } - }).print(); + outputStreamOperator.getSideOutput(new SideOutputTag("ontime")).addSink(resultSink); + outputStreamOperator.getSideOutput(new LateArrivingTag()).addSink(lateResultSink); + see.execute(); + assertEquals(resultSink.getSortedResult(), Arrays.asList("ontime-1", "ontime-2", "ontime-4", "ontime-5")); + //element 4 is handled as timestamp still fall into window + assertEquals(lateResultSink.getSortedResult(), Arrays.asList(3)); } }