diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java index 4154f4945659f..8b088c3d88cef 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.util.Collector; import java.util.Collection; import java.util.List; @@ -34,8 +35,8 @@ */ class StormBoltCollector extends AbstractStormCollector implements IOutputCollector { - /** The Flink output object */ - private final Output flinkOutput; + /** The Flink output Collector */ + private final Collector flinkOutput; /** * Instantiates a new {@link StormBoltCollector} that emits Flink tuples to the given Flink @@ -50,7 +51,7 @@ class StormBoltCollector extends AbstractStormCollector implements IOu * @throws UnsupportedOperationException * if the specified number of attributes is not in the valid range of [0,25] */ - public StormBoltCollector(final int numberOfAttributes, final Output flinkOutput) throws UnsupportedOperationException { + public StormBoltCollector(final int numberOfAttributes, final Collector flinkOutput) throws UnsupportedOperationException { super(numberOfAttributes); assert (flinkOutput != null); this.flinkOutput = flinkOutput; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java index 140e62921b5e0..c7b87ba7459cd 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java @@ -26,6 +26,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; @@ -51,6 +54,12 @@ public class StormBoltWrapper extends AbstractStreamOperator imple /** Number of attributes of the bolt's output tuples */ private final int numberOfAttributes; + /** + * We have to use this because Operators must output + * {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}. + */ + private TimestampedCollector flinkCollector; + /** * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} * such that it can be used within a Flink streaming program. The output type will be one of @@ -93,11 +102,12 @@ public void open(final Configuration parameters) throws Exception { final TopologyContext topologyContext = StormWrapperSetupHelper.convertToTopologyContext( (StreamingRuntimeContext)super.runtimeContext, false); + flinkCollector = new TimestampedCollector(output); OutputCollector stormCollector = null; if (this.numberOfAttributes != -1) { stormCollector = new OutputCollector(new StormBoltCollector( - this.numberOfAttributes, super.output)); + this.numberOfAttributes, flinkCollector)); } this.bolt.prepare(null, topologyContext, stormCollector); @@ -110,8 +120,13 @@ public void close() throws Exception { } @Override - public void processElement(final IN element) throws Exception { - this.bolt.execute(new StormTuple(element)); + public void processElement(final StreamRecord element) throws Exception { + flinkCollector.setTimestamp(element.getTimestamp()); + this.bolt.execute(new StormTuple(element.getValue())); } + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java index 780c75ee67778..dd56c4d44b76c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java @@ -114,9 +114,9 @@ private void testWrapper(final int numberOfAttributes) throws Exception { final StreamRecord record = mock(StreamRecord.class); if (numberOfAttributes == 0) { - when(record.getObject()).thenReturn(rawTuple); + when(record.getValue()).thenReturn(rawTuple); } else { - when(record.getObject()).thenReturn(flinkTuple); + when(record.getValue()).thenReturn(flinkTuple); } final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); @@ -129,8 +129,9 @@ private void testWrapper(final int numberOfAttributes) throws Exception { final StormBoltWrapper wrapper = new StormBoltWrapper(bolt); wrapper.setup(mock(Output.class), taskContext); + wrapper.open(new Configuration()); - wrapper.processElement(record.getObject()); + wrapper.processElement(record); if (numberOfAttributes == 0) { verify(bolt).execute(eq(new StormTuple(rawTuple))); } else { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java index 8885a1b8a58ce..7c91e6f4be162 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java @@ -19,6 +19,8 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; +import org.apache.flink.streaming.api.watermark.Watermark; + import java.util.LinkedList; class TestContext implements SourceContext> { @@ -32,9 +34,23 @@ public void collect(final Tuple1 record) { this.result.add(record.copy()); } + @Override + public void collectWithTimestamp(Tuple1 element, long timestamp) { + this.result.add(element.copy()); + } + + @Override + public void emitWatermark(Watermark mark) { + // ignore it + } + @Override public Object getCheckpointLock() { return null; } + @Override + public void close() { + + } } diff --git a/flink-core/pom.xml b/flink-core/pom.xml index c0c4378411178..d59e755a1f706 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -63,7 +63,8 @@ under the License. guava ${guava.version} - + + diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index 49742956a13dd..b8fa2a2aaa72a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -89,6 +89,10 @@ public class ExecutionConfig implements Serializable { private GlobalJobParameters globalJobParameters = null; + private long autoWatermarkInterval = 0; + + private boolean timestampsEnabled = false; + // Serializers and types registered with Kryo and the PojoSerializer // we store them in lists to ensure they are registered in order in all kryo instances. @@ -140,6 +144,62 @@ public boolean isClosureCleanerEnabled() { return useClosureCleaner; } + /** + * Sets the interval of the automatic watermark emission. Watermaks are used throughout + * the streaming system to keep track of the progress of time. They are used, for example, + * for time based windowing. + * + * @param interval The interval between watermarks in milliseconds. + */ + public ExecutionConfig setAutoWatermarkInterval(long interval) { + this.autoWatermarkInterval = interval; + return this; + } + + /** + * Enables streaming timestamps. When this is enabled all records that are emitted + * from a source have a timestamp attached. This is required if a topology contains + * operations that rely on watermarks and timestamps to perform operations, such as + * event-time windows. + * + *

+ * This is automatically enabled if you enable automatic watermarks. + * + * @see #setAutoWatermarkInterval(long) + */ + public ExecutionConfig enableTimestamps() { + this.timestampsEnabled = true; + return this; + } + + /** + * Disables streaming timestamps. + * + * @see #enableTimestamps() + */ + public ExecutionConfig disableTimestamps() { + this.timestampsEnabled = false; + return this; + } + + /** + * Returns true when timestamps are enabled. + * + * @see #enableTimestamps() + */ + public boolean areTimestampsEnabled() { + return timestampsEnabled; + } + + /** + * Returns the interval of the automatic watermark emission. + * + * @see #setAutoWatermarkInterval(long) + */ + public long getAutoWatermarkInterval() { + return this.autoWatermarkInterval; + } + /** * Gets the parallelism with which operation are executed by default. Operations can * individually override this value to use a specific parallelism. @@ -637,6 +697,8 @@ public String toString() { * getRuntimeContext().getExecutionConfig().getUserConfig() */ public static class GlobalJobParameters implements Serializable { + private static final long serialVersionUID = 1L; + /** * Convert UserConfig into a Map representation. * This can be used by the runtime, for example for presenting the user config in the web frontend. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index 29efc4c4030cf..5bc705d17524f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -114,6 +114,32 @@ public void emit(T record) throws IOException, InterruptedException { } } + /** + * This is used to broadcast Streaming Watermarks in-band with records. This ignores + * the {@link ChannelSelector}. + */ + public void broadcastEmit(T record) throws IOException, InterruptedException { + for (int targetChannel = 0; targetChannel < numChannels; targetChannel++) { + // serialize with corresponding serializer and send full buffer + RecordSerializer serializer = serializers[targetChannel]; + + synchronized (serializer) { + SerializationResult result = serializer.addRecord(record); + while (result.isFullBuffer()) { + Buffer buffer = serializer.getCurrentBuffer(); + + if (buffer != null) { + writer.writeBuffer(buffer, targetChannel); + serializer.clearCurrentBuffer(); + } + + buffer = writer.getBufferProvider().requestBufferBlocking(); + result = serializer.setNextBuffer(buffer); + } + } + } + } + public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedException { for (int targetChannel = 0; targetChannel < numChannels; targetChannel++) { RecordSerializer serializer = serializers[targetChannel]; diff --git a/flink-staging/flink-streaming/flink-streaming-core/pom.xml b/flink-staging/flink-streaming/flink-streaming-core/pom.xml index 6b49770009141..a4eb6a9c62ae2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/pom.xml +++ b/flink-staging/flink-streaming/flink-streaming-core/pom.xml @@ -78,7 +78,14 @@ under the License. guava ${guava.version} - + + + org.apache.commons + commons-math3 + 3.5 + + + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamOutput.java deleted file mode 100644 index aa367ab67c638..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamOutput.java +++ /dev/null @@ -1,90 +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.collector; - -import java.io.IOException; - -import org.apache.flink.runtime.event.task.TaskEvent; -import org.apache.flink.runtime.io.network.api.writer.RecordWriter; -import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.runtime.io.StreamRecordWriter; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.util.Collector; -import org.apache.flink.util.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class StreamOutput implements Collector { - - private static final Logger LOG = LoggerFactory.getLogger(StreamOutput.class); - - private RecordWriter>> output; - private SerializationDelegate> serializationDelegate; - private StreamRecord streamRecord; - - public StreamOutput(RecordWriter>> output, SerializationDelegate> serializationDelegate) { - - this.serializationDelegate = serializationDelegate; - - if (serializationDelegate != null) { - this.streamRecord = serializationDelegate.getInstance(); - } else { - throw new RuntimeException("Serializer cannot be null"); - } - this.output = output; - } - - public RecordWriter>> getRecordWriter() { - return output; - } - - @Override - public void collect(OUT record) { - streamRecord.setObject(record); - serializationDelegate.setInstance(streamRecord); - - try { - output.emit(serializationDelegate); - } catch (Exception e) { - if (LOG.isErrorEnabled()) { - LOG.error("Emit failed due to: {}", StringUtils.stringifyException(e)); - } - } - } - - @Override - public void close() { - if (output instanceof StreamRecordWriter) { - ((StreamRecordWriter>>) output).close(); - } else { - try { - output.flush(); - } catch (IOException e) { - e.printStackTrace(); - } - } - } - - public void clearBuffers() { - output.clearBuffers(); - } - - public void broadcastEvent(TaskEvent barrier) throws IOException, InterruptedException { - output.broadcastEvent(barrier); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java index b90cce2e6d0ad..0fe84d8ecbe61 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java @@ -21,25 +21,27 @@ import java.util.List; import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; public class BroadcastOutputSelectorWrapper implements OutputSelectorWrapper { private static final long serialVersionUID = 1L; - private List> outputs; + private List>> outputs; public BroadcastOutputSelectorWrapper() { - outputs = new ArrayList>(); + outputs = new ArrayList>>(); } @SuppressWarnings("unchecked") @Override - public void addCollector(Collector output, StreamEdge edge) { - outputs.add((Collector) output); + public void addCollector(Collector> output, StreamEdge edge) { + Collector output1 = output; + outputs.add((Collector>) output1); } @Override - public Iterable> getSelectedOutputs(OUT record) { + public Iterable>> getSelectedOutputs(OUT record) { return outputs; } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java index 8ca05089f0b5a..46b315de6a483 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java @@ -25,6 +25,7 @@ import java.util.Set; import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,31 +38,32 @@ public class DirectedOutputSelectorWrapper implements OutputSelectorWrapper private List> outputSelectors; - private Map>> outputMap; - private Set> selectAllOutputs; + private Map>>> outputMap; + private Set>> selectAllOutputs; public DirectedOutputSelectorWrapper(List> outputSelectors) { this.outputSelectors = outputSelectors; - this.selectAllOutputs = new HashSet>(); //new LinkedList>(); - this.outputMap = new HashMap>>(); + this.selectAllOutputs = new HashSet>>(); //new LinkedList>(); + this.outputMap = new HashMap>>>(); } @SuppressWarnings("unchecked") @Override - public void addCollector(Collector output, StreamEdge edge) { + public void addCollector(Collector> output, StreamEdge edge) { + Collector output1 = output; List selectedNames = edge.getSelectedNames(); if (selectedNames.isEmpty()) { - selectAllOutputs.add((Collector) output); + selectAllOutputs.add((Collector>) output1); } else { for (String selectedName : selectedNames) { if (!outputMap.containsKey(selectedName)) { - outputMap.put(selectedName, new LinkedList>()); - outputMap.get(selectedName).add((Collector) output); + outputMap.put(selectedName, new LinkedList>>()); + outputMap.get(selectedName).add((Collector>) output1); } else { if (!outputMap.get(selectedName).contains(output)) { - outputMap.get(selectedName).add((Collector) output); + outputMap.get(selectedName).add((Collector>) output1); } } } @@ -69,14 +71,14 @@ public void addCollector(Collector output, StreamEdge edge) { } @Override - public Iterable> getSelectedOutputs(OUT record) { - Set> selectedOutputs = new HashSet>(selectAllOutputs); + public Iterable>> getSelectedOutputs(OUT record) { + Set>> selectedOutputs = new HashSet>>(selectAllOutputs); for (OutputSelector outputSelector : outputSelectors) { Iterable outputNames = outputSelector.select(record); for (String outputName : outputNames) { - List> outputList = outputMap.get(outputName); + List>> outputList = outputMap.get(outputName); try { selectedOutputs.addAll(outputList); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java index 937b69f58ea7c..9133ac0a310cc 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java @@ -20,12 +20,13 @@ import java.io.Serializable; import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; public interface OutputSelectorWrapper extends Serializable { - public void addCollector(Collector output, StreamEdge edge); + public void addCollector(Collector> output, StreamEdge edge); - public Iterable> getSelectedOutputs(OUT record); + public Iterable>> getSelectedOutputs(OUT record); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/EventTimeSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/EventTimeSourceFunction.java new file mode 100644 index 0000000000000..ab380d74ec052 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/EventTimeSourceFunction.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.api.functions.source; + + +/** + * A marker interface that must be implemented by {@link SourceFunction}s that emit elements with + * timestamps. The {@link SourceFunction} can extract the timestamp from the data and attach it to + * the element upon emission. + * + *

+ * Event-time sources must manually emit + * {@link org.apache.flink.streaming.api.watermark.Watermark watermarks} to keep track of progress. + * Automatic emission of watermarks will be suppressed if a source implements this interface. + * + *

+ * Elements must be emitted using + * {@link SourceFunction.SourceContext#collectWithTimestamp(Object, long)} + * and watermarks can be emitted using + * {@link SourceFunction.SourceContext#emitWatermark(org.apache.flink.streaming.api.watermark.Watermark)}. + * + * @param Type of the elements emitted by this source. + */ +public interface EventTimeSourceFunction extends SourceFunction { } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java index 58ee1daebf1dc..886d6e784e8f8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java @@ -18,9 +18,10 @@ package org.apache.flink.streaming.api.functions.source; -import java.io.Serializable; - import org.apache.flink.api.common.functions.Function; +import org.apache.flink.streaming.api.watermark.Watermark; + +import java.io.Serializable; /** * Base interface for all stream data sources in Flink. The contract of a stream source @@ -28,9 +29,10 @@ * is called with a {@link org.apache.flink.util.Collector} that can be used for emitting elements. * The run method can run for as long as necessary. The source must, however, react to an * invocation of {@link #cancel} by breaking out of its main loop. - * - * Note about checkpointed sources + * *

+ * Note about checkpointed sources
+ * * Sources that also implement the {@link org.apache.flink.streaming.api.checkpoint.Checkpointed} * interface must ensure that state checkpointing, updating of internal state and emission of * elements are not done concurrently. This is achieved by using the provided checkpointing lock @@ -41,7 +43,6 @@ * This is the basic pattern one should follow when implementing a (checkpointed) source: *

* - *
  * {@code
  *  public class ExampleSource implements SourceFunction, Checkpointed {
  *      private long count = 0L;
@@ -70,6 +71,14 @@
  * }
  * 
* + * + *

+ * Note about element timestamps and watermarks:
+ * Sources must only manually emit watermarks when they implement + * {@link EventTimeSourceFunction }. + * Otherwise, elements automatically get the current timestamp assigned at ingress + * and the system automatically emits watermarks. + * * @param The type of the elements produced by this source. */ public interface SourceFunction extends Function, Serializable { @@ -106,18 +115,48 @@ public interface SourceFunction extends Function, Serializable { public static interface SourceContext { /** - * Emits one element from the source. - * - * @param element The element to emit. + * Emits one element from the source. The result of {@link System#currentTimeMillis()} is set as + * the timestamp of the emitted element. + * + * @param element The element to emit */ void collect(T element); + /** + * Emits one element from the source with the given timestamp. + * + * @param element The element to emit + * @param timestamp The timestamp in milliseconds + */ + public void collectWithTimestamp(T element, long timestamp); + + /** + * Emits the given {@link org.apache.flink.streaming.api.watermark.Watermark}. + * + *

+ * Important: + * Sources must only manually emit watermarks when they implement + * {@link EventTimeSourceFunction}. + * Otherwise, elements automatically get the current timestamp assigned at ingress + * and the system automatically emits watermarks. + * + * @param mark The {@link Watermark} to emit + */ + void emitWatermark(Watermark mark); + + /** * Returns the checkpoint lock. Please refer to the explanation about checkpointed sources * in {@link org.apache.flink.streaming.api.functions.source.SourceFunction}. * - * @return The object to use the lock. + * @return The object to use as the lock. */ Object getCheckpointLock(); + + /** + * This must be called when closing the source operator to allow the {@link SourceContext} + * to clean up internal state. + */ + void close(); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 6a44104c38fe6..d0e806425992b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -24,12 +24,12 @@ import java.util.List; import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.runtime.tasks.StreamTaskException; import org.apache.flink.util.InstantiationUtil; @@ -94,26 +94,26 @@ public String getOperatorName() { return config.getString(OPERATOR_NAME, "Missing"); } - public void setTypeSerializerIn1(StreamRecordSerializer serializer) { + public void setTypeSerializerIn1(TypeSerializer serializer) { setTypeSerializer(TYPE_SERIALIZER_IN_1, serializer); } - public void setTypeSerializerIn2(StreamRecordSerializer serializer) { + public void setTypeSerializerIn2(TypeSerializer serializer) { setTypeSerializer(TYPE_SERIALIZER_IN_2, serializer); } - public void setTypeSerializerOut1(StreamRecordSerializer serializer) { + public void setTypeSerializerOut1(TypeSerializer serializer) { setTypeSerializer(TYPE_SERIALIZER_OUT_1, serializer); } - public void setTypeSerializerOut2(StreamRecordSerializer serializer) { + public void setTypeSerializerOut2(TypeSerializer serializer) { setTypeSerializer(TYPE_SERIALIZER_OUT_2, serializer); } @SuppressWarnings("unchecked") - public StreamRecordSerializer getTypeSerializerIn1(ClassLoader cl) { + public TypeSerializer getTypeSerializerIn1(ClassLoader cl) { try { - return (StreamRecordSerializer) InstantiationUtil.readObjectFromConfig(this.config, + return (TypeSerializer) InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_IN_1, cl); } catch (Exception e) { throw new StreamTaskException("Could not instantiate serializer.", e); @@ -121,9 +121,9 @@ public StreamRecordSerializer getTypeSerializerIn1(ClassLoader cl) { } @SuppressWarnings("unchecked") - public StreamRecordSerializer getTypeSerializerIn2(ClassLoader cl) { + public TypeSerializer getTypeSerializerIn2(ClassLoader cl) { try { - return (StreamRecordSerializer) InstantiationUtil.readObjectFromConfig(this.config, + return (TypeSerializer) InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_IN_2, cl); } catch (Exception e) { throw new StreamTaskException("Could not instantiate serializer.", e); @@ -131,9 +131,9 @@ public StreamRecordSerializer getTypeSerializerIn2(ClassLoader cl) { } @SuppressWarnings("unchecked") - public StreamRecordSerializer getTypeSerializerOut1(ClassLoader cl) { + public TypeSerializer getTypeSerializerOut1(ClassLoader cl) { try { - return (StreamRecordSerializer) InstantiationUtil.readObjectFromConfig(this.config, + return (TypeSerializer) InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_OUT_1, cl); } catch (Exception e) { throw new StreamTaskException("Could not instantiate serializer.", e); @@ -141,16 +141,16 @@ public StreamRecordSerializer getTypeSerializerOut1(ClassLoader cl) { } @SuppressWarnings("unchecked") - public StreamRecordSerializer getTypeSerializerOut2(ClassLoader cl) { + public TypeSerializer getTypeSerializerOut2(ClassLoader cl) { try { - return (StreamRecordSerializer) InstantiationUtil.readObjectFromConfig(this.config, + return (TypeSerializer) InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_OUT_2, cl); } catch (Exception e) { throw new StreamTaskException("Could not instantiate serializer.", e); } } - private void setTypeSerializer(String key, StreamRecordSerializer typeWrapper) { + private void setTypeSerializer(String key, TypeSerializer typeWrapper) { try { InstantiationUtil.writeObjectToConfig(typeWrapper, this.config, key); } catch (IOException e) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index 64c349e660173..f1428b4ed18b2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -34,6 +34,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.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.MissingTypeInfo; @@ -49,7 +50,6 @@ import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; import org.apache.flink.streaming.runtime.tasks.SourceStreamTask; import org.apache.flink.streaming.runtime.tasks.StreamIterationHead; @@ -168,12 +168,9 @@ public void addOperator(Integer vertexID, StreamOperator operator addNode(vertexID, OneInputStreamTask.class, operatorObject, operatorName); } - StreamRecordSerializer inSerializer = inTypeInfo != null ? new StreamRecordSerializer( - inTypeInfo, executionConfig) : null; + TypeSerializer inSerializer = inTypeInfo != null && !(inTypeInfo instanceof MissingTypeInfo) ? inTypeInfo.createSerializer(executionConfig) : null; - StreamRecordSerializer outSerializer = (outTypeInfo != null) - && !(outTypeInfo instanceof MissingTypeInfo) ? new StreamRecordSerializer( - outTypeInfo, executionConfig) : null; + TypeSerializer outSerializer = outTypeInfo != null && !(outTypeInfo instanceof MissingTypeInfo) ? outTypeInfo.createSerializer(executionConfig) : null; setSerializers(vertexID, inSerializer, null, outSerializer); @@ -183,18 +180,15 @@ public void addOperator(Integer vertexID, StreamOperator operator } public void addCoOperator(Integer vertexID, - TwoInputStreamOperator taskoperatorObject, - TypeInformation in1TypeInfo, TypeInformation in2TypeInfo, - TypeInformation outTypeInfo, String operatorName) { + TwoInputStreamOperator taskoperatorObject, TypeInformation in1TypeInfo, + TypeInformation in2TypeInfo, TypeInformation outTypeInfo, String operatorName) { addNode(vertexID, TwoInputStreamTask.class, taskoperatorObject, operatorName); - StreamRecordSerializer outSerializer = (outTypeInfo != null) - && !(outTypeInfo instanceof MissingTypeInfo) ? new StreamRecordSerializer( - outTypeInfo, executionConfig) : null; + TypeSerializer outSerializer = (outTypeInfo != null) && !(outTypeInfo instanceof MissingTypeInfo) ? + outTypeInfo.createSerializer(executionConfig) : null; - setSerializers(vertexID, new StreamRecordSerializer(in1TypeInfo, executionConfig), - new StreamRecordSerializer(in2TypeInfo, executionConfig), outSerializer); + setSerializers(vertexID, in1TypeInfo.createSerializer(executionConfig), in2TypeInfo.createSerializer(executionConfig), outSerializer); if (LOG.isDebugEnabled()) { LOG.debug("CO-TASK: {}", vertexID); @@ -316,8 +310,7 @@ public void finalizeLoops() { // We set the proper serializers for the sink/source setSerializersFrom(tailOps.get(0).getId(), sink.getId()); if (loop.isCoIteration()) { - source.setSerializerOut(new StreamRecordSerializer(loop - .getFeedbackType(), executionConfig)); + source.setSerializerOut(loop.getFeedbackType().createSerializer(executionConfig)); } else { setSerializersFrom(headOpsInGroup.get(0).getId(), source.getId()); } @@ -430,8 +423,7 @@ public void setBufferTimeout(Integer vertexID, long bufferTimeout) { getStreamNode(vertexID).setBufferTimeout(bufferTimeout); } - private void setSerializers(Integer vertexID, StreamRecordSerializer in1, - StreamRecordSerializer in2, StreamRecordSerializer out) { + private void setSerializers(Integer vertexID, TypeSerializer in1, TypeSerializer in2, TypeSerializer out) { StreamNode vertex = getStreamNode(vertexID); vertex.setSerializerIn1(in1); vertex.setSerializerIn2(in2); @@ -447,9 +439,7 @@ private void setSerializersFrom(Integer from, Integer to) { } public void setOutType(Integer vertexID, TypeInformation outType) { - StreamRecordSerializer serializer = new StreamRecordSerializer(outType, - executionConfig); - getStreamNode(vertexID).setSerializerOut(serializer); + getStreamNode(vertexID).setSerializerOut(outType.createSerializer(executionConfig)); } public void setOperator(Integer vertexID, StreamOperator operatorObject) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java index 0b909bd0572d4..62e2d839f52b8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.flink.api.common.io.InputFormat; +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; @@ -29,7 +30,6 @@ import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapperFactory; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; /** * Class representing the operators in the streaming programs, with all their @@ -53,9 +53,9 @@ public class StreamNode implements Serializable { private transient StreamOperator operator; private List> outputSelectors; - private StreamRecordSerializer typeSerializerIn1; - private StreamRecordSerializer typeSerializerIn2; - private StreamRecordSerializer typeSerializerOut; + private TypeSerializer typeSerializerIn1; + private TypeSerializer typeSerializerIn2; + private TypeSerializer typeSerializerOut; private List inEdges = new ArrayList(); private List outEdges = new ArrayList(); @@ -168,27 +168,27 @@ public void addOutputSelector(OutputSelector outputSelector) { this.outputSelectors.add(outputSelector); } - public StreamRecordSerializer getTypeSerializerIn1() { + public TypeSerializer getTypeSerializerIn1() { return typeSerializerIn1; } - public void setSerializerIn1(StreamRecordSerializer typeSerializerIn1) { + public void setSerializerIn1(TypeSerializer typeSerializerIn1) { this.typeSerializerIn1 = typeSerializerIn1; } - public StreamRecordSerializer getTypeSerializerIn2() { + public TypeSerializer getTypeSerializerIn2() { return typeSerializerIn2; } - public void setSerializerIn2(StreamRecordSerializer typeSerializerIn2) { + public void setSerializerIn2(TypeSerializer typeSerializerIn2) { this.typeSerializerIn2 = typeSerializerIn2; } - public StreamRecordSerializer getTypeSerializerOut() { + public TypeSerializer getTypeSerializerOut() { return typeSerializerOut; } - public void setSerializerOut(StreamRecordSerializer typeSerializerOut) { + public void setSerializerOut(TypeSerializer typeSerializerOut) { this.typeSerializerOut = typeSerializerOut; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index cb4bcb0fb9fe6..3956d758a3014 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; /** @@ -35,7 +36,7 @@ public abstract class AbstractStreamOperator implements StreamOperator protected transient ExecutionConfig executionConfig; - public transient Output output; + public transient Output> output; protected boolean inputCopyDisabled = false; @@ -43,7 +44,7 @@ public abstract class AbstractStreamOperator implements StreamOperator protected ChainingStrategy chainingStrategy = ChainingStrategy.HEAD; @Override - public void setup(Output output, StreamingRuntimeContext runtimeContext) { + public void setup(Output> output, StreamingRuntimeContext runtimeContext) { this.output = output; this.executionConfig = runtimeContext.getExecutionConfig(); this.runtimeContext = runtimeContext; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java index b2d9c91004c66..23c4ab87505c9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.api.state.StreamOperatorState; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; @@ -57,7 +58,7 @@ public AbstractUdfStreamOperator(F userFunction) { } @Override - public void setup(Output output, StreamingRuntimeContext runtimeContext) { + public final void setup(Output> output, StreamingRuntimeContext runtimeContext) { super.setup(output, runtimeContext); FunctionUtils.setFunctionRuntimeContext(userFunction, runtimeContext); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java index d24ffed30bc69..7ca540f7d16d6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java @@ -18,6 +18,9 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + /** * Interface for stream operators with one input. Use * {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator} as a base class if @@ -27,5 +30,18 @@ * @param The output type of the operator */ public interface OneInputStreamOperator extends StreamOperator { - public void processElement(IN element) throws Exception; + + /** + * Processes one element that arrived at this operator. + * This method is guaranteed to not be called concurrently with other methods of the operator. + */ + public void processElement(StreamRecord element) throws Exception; + + /** + * Processes a {@link Watermark}. + * This method is guaranteed to not be called concurrently with other methods of the operator. + * + * @see org.apache.flink.streaming.api.watermark.Watermark + */ + public void processWatermark(Watermark mark) throws Exception; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java index d6f810adc3fa5..89d5560691b53 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java @@ -17,16 +17,25 @@ */ package org.apache.flink.streaming.api.operators; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.util.Collector; /** * A {@link org.apache.flink.streaming.api.operators.StreamOperator} is supplied with an object * of this interface that can be used to emit elements and other messages, such as barriers - * and low watermarks, from an operator. + * and watermarks, from an operator. * * @param The type of the elments that can be emitted. */ public interface Output extends Collector { - // NOTE: This does not yet have methods for barriers/low watermarks, this needs to be - // extended when this functionality arrives. + + /** + * Emits a {@link Watermark} from an operator. This watermark is broadcast to all downstream + * operators. + * + *

+ * A watermark specifies that no element with a timestamp older or equal to the watermark + * timestamp will be emitted in the future. + */ + void emitWatermark(Watermark mark); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamCounter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamCounter.java index 240e2b1aba310..efe5d527f9bd2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamCounter.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamCounter.java @@ -17,6 +17,9 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + public class StreamCounter extends AbstractStreamOperator implements OneInputStreamOperator { private static final long serialVersionUID = 1L; @@ -28,7 +31,12 @@ public StreamCounter() { } @Override - public void processElement(IN element) { - output.collect(++count); + public void processElement(StreamRecord element) { + output.collect(element.replace(++count)); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFilter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFilter.java index a54a4eaece8cc..2ff220e80cd8e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFilter.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFilter.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamFilter extends AbstractUdfStreamOperator> implements OneInputStreamOperator { @@ -29,9 +31,14 @@ public StreamFilter(FilterFunction filterFunction) { } @Override - public void processElement(IN element) throws Exception { - if (userFunction.filter(element)) { + public void processElement(StreamRecord element) throws Exception { + if (userFunction.filter(element.getValue())) { output.collect(element); } } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java index e8da2c7ee2c70..5547c6ac46e31 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java @@ -18,6 +18,9 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamFlatMap extends AbstractUdfStreamOperator> @@ -25,13 +28,27 @@ public class StreamFlatMap private static final long serialVersionUID = 1L; + private TimestampedCollector collector; + public StreamFlatMap(FlatMapFunction flatMapper) { super(flatMapper); chainingStrategy = ChainingStrategy.ALWAYS; } @Override - public void processElement(IN element) throws Exception { - userFunction.flatMap(element, output); + public void open(Configuration parameters) throws Exception { + super.open(parameters); + collector = new TimestampedCollector(output); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + collector.setTimestamp(element.getTimestamp()); + userFunction.flatMap(element.getValue(), collector); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFold.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFold.java index 580477a7c3f44..a5e526408b9d2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFold.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFold.java @@ -21,6 +21,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamFold extends AbstractUdfStreamOperator> @@ -40,9 +42,9 @@ public StreamFold(FoldFunction folder, OUT initialValue, TypeInformatio } @Override - public void processElement(IN element) throws Exception { - accumulator = userFunction.fold(outTypeSerializer.copy(accumulator), element); - output.collect(accumulator); + public void processElement(StreamRecord element) throws Exception { + accumulator = userFunction.fold(outTypeSerializer.copy(accumulator), element.getValue()); + output.collect(element.replace(accumulator)); } @Override @@ -50,4 +52,9 @@ public void open(Configuration config) throws Exception { super.open(config); this.outTypeSerializer = outTypeInformation.createSerializer(executionConfig); } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java index 08107a926eb39..5272a48eb8d04 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.functions.FoldFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamGroupedFold extends StreamFold { @@ -41,19 +42,18 @@ public StreamGroupedFold(FoldFunction folder, KeySelector keySel } @Override - public void processElement(IN element) throws Exception { - Object key = keySelector.getKey(element); + public void processElement(StreamRecord element) throws Exception { + Object key = keySelector.getKey(element.getValue()); OUT accumulator = values.get(key); - FoldFunction folder = ((FoldFunction) userFunction); if (accumulator != null) { - OUT folded = folder.fold(outTypeSerializer.copy(accumulator), element); + OUT folded = userFunction.fold(outTypeSerializer.copy(accumulator), element.getValue()); values.put(key, folded); - output.collect(folded); + output.collect(element.replace(folded)); } else { - OUT first = folder.fold(outTypeSerializer.copy(initialValue), element); + OUT first = userFunction.fold(outTypeSerializer.copy(initialValue), element.getValue()); values.put(key, first); - output.collect(first); + output.collect(element.replace(first)); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java index 8269be7128677..6be011e7c4aa8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamGroupedReduce extends StreamReduce { @@ -37,17 +38,17 @@ public StreamGroupedReduce(ReduceFunction reducer, KeySelector keySel } @Override - public void processElement(IN element) throws Exception { - Object key = keySelector.getKey(element); + public void processElement(StreamRecord element) throws Exception { + Object key = keySelector.getKey(element.getValue()); IN currentValue = values.get(key); if (currentValue != null) { // TODO: find a way to let operators copy elements (maybe) - IN reduced = userFunction.reduce(currentValue, element); + IN reduced = userFunction.reduce(currentValue, element.getValue()); values.put(key, reduced); - output.collect(reduced); + output.collect(element.replace(reduced)); } else { - values.put(key, element); - output.collect(element); + values.put(key, element.getValue()); + output.collect(element.replace(element.getValue())); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamMap.java index 08dc9813af015..7d5c7cc99912c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamMap.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamMap.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamMap extends AbstractUdfStreamOperator> @@ -31,7 +33,12 @@ public StreamMap(MapFunction mapper) { } @Override - public void processElement(IN element) throws Exception { - output.collect(userFunction.map(element)); + public void processElement(StreamRecord element) throws Exception { + output.collect(element.replace(userFunction.map(element.getValue()))); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java index 75cea5fb72fc8..9dd18b2f3cb7c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java @@ -20,6 +20,7 @@ import java.io.Serializable; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; /** @@ -37,7 +38,7 @@ public interface StreamOperator extends Serializable { /** * Initializes the {@link StreamOperator} for input and output handling. */ - public void setup(Output output, StreamingRuntimeContext runtimeContext); + public void setup(Output> output, StreamingRuntimeContext runtimeContext); /** * This method is called before any elements are processed. diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java index 83613d83d9e7e..c0815b521a408 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamProject extends AbstractStreamOperator @@ -43,11 +45,11 @@ public StreamProject(int[] fields, TypeSerializer outSerializer) { @Override - public void processElement(IN element) throws Exception { + public void processElement(StreamRecord element) throws Exception { for (int i = 0; i < this.numFields; i++) { - outTuple.setField(((Tuple) element).getField(fields[i]), i); + outTuple.setField(((Tuple) element.getValue()).getField(fields[i]), i); } - output.collect(outTuple); + output.collect(element.replace(outTuple)); } @Override @@ -55,4 +57,9 @@ public void open(Configuration config) throws Exception { super.open(config); outTuple = outSerializer.createInstance(); } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamReduce.java index 97cebc18e0bc2..52c07d06d0856 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamReduce.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamReduce.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamReduce extends AbstractUdfStreamOperator> implements OneInputStreamOperator { @@ -34,15 +36,19 @@ public StreamReduce(ReduceFunction reducer) { } @Override - public void processElement(IN element) throws Exception { + public void processElement(StreamRecord element) throws Exception { if (currentValue != null) { - // TODO: give operator a way to specify that elements should be copied - currentValue = userFunction.reduce(currentValue, element); + currentValue = userFunction.reduce(currentValue, element.getValue()); } else { - currentValue = element; + currentValue = element.getValue(); } - output.collect(currentValue); + output.collect(element.replace(currentValue)); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java index 5399302005083..6961a4dc848a4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class StreamSink extends AbstractUdfStreamOperator> implements OneInputStreamOperator { @@ -31,7 +33,12 @@ public StreamSink(SinkFunction sinkFunction) { } @Override - public void processElement(IN element) throws Exception { - userFunction.invoke(element); + public void processElement(StreamRecord element) throws Exception { + userFunction.invoke(element.getValue()); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + // ignore it for now, we are a sink, after all } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java index 907f93ad8667c..0cc46f549b236 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java @@ -17,39 +17,274 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.util.Collector; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; /** * {@link StreamOperator} for streaming sources. */ -public class StreamSource extends AbstractUdfStreamOperator> implements StreamOperator { +public class StreamSource extends AbstractUdfStreamOperator> implements StreamOperator { private static final long serialVersionUID = 1L; - public StreamSource(SourceFunction sourceFunction) { + public StreamSource(SourceFunction sourceFunction) { super(sourceFunction); this.chainingStrategy = ChainingStrategy.HEAD; } - public void run(final Object lockingObject, final Collector collector) throws Exception { - SourceFunction.SourceContext ctx = new SourceFunction.SourceContext() { - @Override - public void collect(OUT element) { - collector.collect(element); - } + public void run(final Object lockingObject, final Output> collector) throws Exception { - @Override - public Object getCheckpointLock() { - return lockingObject; - } - }; + SourceFunction.SourceContext ctx = null; + if (userFunction instanceof EventTimeSourceFunction) { + ctx = new ManualWatermarkContext(lockingObject, collector); + } else if (executionConfig.getAutoWatermarkInterval() > 0) { + ctx = new AutomaticWatermarkContext(lockingObject, collector, executionConfig); + } else if (executionConfig.areTimestampsEnabled()) { + ctx = new NonTimestampContext(lockingObject, collector); + } else { + ctx = new NonWatermarkContext(lockingObject, collector); + } userFunction.run(ctx); } public void cancel() { + userFunction.cancel(); } + + /** + * {@link SourceFunction.SourceContext} to be used for sources that don't emit watermarks. + * In addition to {@link NonWatermarkContext} this will also not attach timestamps to sources. + * (Technically it will always set the timestamp to 0). + */ + public static class NonTimestampContext implements SourceFunction.SourceContext { + + + private final Object lockingObject; + private final Output> output; + StreamRecord reuse; + + public NonTimestampContext(Object lockingObjectParam, Output> outputParam) { + this.lockingObject = lockingObjectParam; + this.output = outputParam; + this.reuse = new StreamRecord(null); + } + + @Override + public void collect(T element) { + output.collect(reuse.replace(element, 0)); + } + + @Override + public void collectWithTimestamp(T element, long timestamp) { + throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" + + " elements with a timestamp. See interface ManualTimestampSourceFunction" + + " if you want to manually assign timestamps to elements."); + } + + @Override + public void emitWatermark(Watermark mark) { + throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" + + " elements with a timestamp. See interface ManualTimestampSourceFunction" + + " if you want to manually assign timestamps to elements."); + } + + @Override + public Object getCheckpointLock() { + return lockingObject; + } + + @Override + public void close() { + } + } + + /** + * {@link SourceFunction.SourceContext} to be used for sources that don't emit watermarks. + */ + public static class NonWatermarkContext implements SourceFunction.SourceContext { + + + private final Object lockingObject; + private final Output> output; + StreamRecord reuse; + + public NonWatermarkContext(Object lockingObjectParam, Output> outputParam) { + this.lockingObject = lockingObjectParam; + this.output = outputParam; + this.reuse = new StreamRecord(null); + } + + @Override + public void collect(T element) { + long currentTime = System.currentTimeMillis(); + output.collect(reuse.replace(element, currentTime)); + } + + @Override + public void collectWithTimestamp(T element, long timestamp) { + throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" + + " elements with a timestamp. See interface ManualTimestampSourceFunction" + + " if you want to manually assign timestamps to elements."); + } + + @Override + public void emitWatermark(Watermark mark) { + throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" + + " elements with a timestamp. See interface ManualTimestampSourceFunction" + + " if you want to manually assign timestamps to elements."); + } + + @Override + public Object getCheckpointLock() { + return lockingObject; + } + + @Override + public void close() { + } + } + + /** + * {@link SourceFunction.SourceContext} to be used for sources with automatic timestamps + * and watermark emission. + */ + public static class AutomaticWatermarkContext implements SourceFunction.SourceContext { + + private transient ScheduledFuture watermarkTimer = null; + private final long watermarkInterval; + + private final Object lockingObject; + private final Output> output; + StreamRecord reuse; + + private volatile long lastWatermarkTime; + + public AutomaticWatermarkContext(Object lockingObjectParam, + Output> outputParam, + ExecutionConfig executionConfig) { + this.lockingObject = lockingObjectParam; + this.output = outputParam; + this.reuse = new StreamRecord(null); + + watermarkInterval = executionConfig.getAutoWatermarkInterval(); + + ScheduledExecutorService service = Executors.newScheduledThreadPool(2); + + watermarkTimer = service.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + long currentTime = System.currentTimeMillis(); + // align the watermarks across all machines. this will ensure that we + // don't have watermarks that creep along at different intervals because + // the machine clocks are out of sync + long watermarkTime = currentTime - (currentTime % watermarkInterval); + if (watermarkTime - lastWatermarkTime >= watermarkInterval) { + synchronized (lockingObject) { + if (watermarkTime - lastWatermarkTime >= watermarkInterval) { + output.emitWatermark(new Watermark(watermarkTime)); + lastWatermarkTime = watermarkTime; + } + } + } + } + }, 0, watermarkInterval, TimeUnit.MILLISECONDS); + + } + + @Override + public void collect(T element) { + synchronized (lockingObject) { + long currentTime = System.currentTimeMillis(); + output.collect(reuse.replace(element, currentTime)); + + long watermarkTime = currentTime - (currentTime % watermarkInterval); + if (watermarkTime - lastWatermarkTime >= watermarkInterval) { + output.emitWatermark(new Watermark(watermarkTime)); + lastWatermarkTime = watermarkTime; + } + } + } + + @Override + public void collectWithTimestamp(T element, long timestamp) { + throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" + + " elements with a timestamp. See interface ManualTimestampSourceFunction" + + " if you want to manually assign timestamps to elements."); + } + + @Override + public void emitWatermark(Watermark mark) { + throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" + + " elements with a timestamp. See interface ManualTimestampSourceFunction" + + " if you want to manually assign timestamps to elements."); + } + + @Override + public Object getCheckpointLock() { + return lockingObject; + } + + @Override + public void close() { + if (watermarkTimer != null && !watermarkTimer.isDone()) { + watermarkTimer.cancel(true); + } + } + } + + /** + * {@link SourceFunction.SourceContext} to be used for sources with manual timestamp + * assignment and manual watermark emission. + */ + public static class ManualWatermarkContext implements SourceFunction.SourceContext { + + private final Object lockingObject; + private final Output> output; + StreamRecord reuse; + + public ManualWatermarkContext(Object lockingObject, Output> output) { + this.lockingObject = lockingObject; + this.output = output; + this.reuse = new StreamRecord(null); + } + + @Override + public void collect(T element) { + throw new UnsupportedOperationException("Manual-Timestamp sources can only emit" + + " elements with a timestamp. Please use collectWithTimestamp()."); + } + + @Override + public void collectWithTimestamp(T element, long timestamp) { + synchronized (lockingObject) { + output.collect(reuse.replace(element, timestamp)); + } + } + + @Override + public void emitWatermark(Watermark mark) { + output.emitWatermark(mark); + } + + @Override + public Object getCheckpointLock() { + return lockingObject; + } + + @Override + public void close() { + + } + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java new file mode 100644 index 0000000000000..0ff223c0f86fc --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java @@ -0,0 +1,63 @@ +/** + * 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.operators; + +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +/** + * Wrapper around an {@link Output} for user functions that expect a {@link Collector}. + * Before giving the {@link TimestampedCollector} to a user function you must set + * the {@link Instant timestamp} that should be attached to emitted elements. Most operators + * would set the {@link Instant timestamp} of the incoming {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord} here. + * + * @param The type of the elments that can be emitted. + */ +public class TimestampedCollector implements Collector { + private final Output> output; + private long timestamp; + private StreamRecord reuse; + + /** + * Creates a new {@link TimestampedCollector} that wraps the given {@link Output}. + */ + public TimestampedCollector(Output> output) { + this.output = output; + this.reuse = new StreamRecord(null); + } + + @Override + public void collect(T record) { + output.collect(reuse.replace(record, timestamp)); + } + + /** + * Sets the {@link Instant timestamp} that is attached to elements that get emitted using + * {@link #collect} + * @param timestamp The timestamp in milliseconds + */ + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + @Override + public void close() { + output.close(); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java index 2b3090b69bbbd..afc6d1bbc11b8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java @@ -18,6 +18,9 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + /** * Interface for stream operators with two inputs. Use * {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator} as a base class if @@ -29,7 +32,32 @@ */ public interface TwoInputStreamOperator extends StreamOperator { - public void processElement1(IN1 element) throws Exception; + /** + * Processes one element that arrived on the first input of this two-input operator. + * This method is guaranteed to not be called concurrently with other methods of the operator. + */ + public void processElement1(StreamRecord element) throws Exception; + + /** + * Processes one element that arrived on the second input of this two-input operator. + * This method is guaranteed to not be called concurrently with other methods of the operator. + */ + public void processElement2(StreamRecord element) throws Exception; + + /** + * Processes a {@link Watermark} that arrived on the first input of this two-input operator. + * This method is guaranteed to not be called concurrently with other methods of the operator. + * + * @see org.apache.flink.streaming.api.watermark.Watermark + */ + public void processWatermark1(Watermark mark) throws Exception; + + /** + * Processes a {@link Watermark} that arrived on the second input of this two-input operator. + * This method is guaranteed to not be called concurrently with other methods of the operator. + * + * @see org.apache.flink.streaming.api.watermark.Watermark + */ + public void processWatermark2(Watermark mark) throws Exception; - public void processElement2(IN2 element) throws Exception; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java index e3662d620a5c2..d2bd107d55cfd 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java @@ -17,9 +17,13 @@ package org.apache.flink.streaming.api.operators.co; +import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class CoStreamFlatMap extends AbstractUdfStreamOperator> @@ -27,18 +31,54 @@ public class CoStreamFlatMap private static final long serialVersionUID = 1L; + private transient TimestampedCollector collector; + + // We keep track of watermarks from both inputs, the combined input is the minimum + // Once the minimum advances we emit a new watermark for downstream operators + private long combinedWatermark = Long.MIN_VALUE; + private long input1Watermark = Long.MAX_VALUE; + private long input2Watermark = Long.MAX_VALUE; + public CoStreamFlatMap(CoFlatMapFunction flatMapper) { super(flatMapper); } @Override - public void processElement1(IN1 element) throws Exception { - userFunction.flatMap1(element, output); + public void open(Configuration parameters) throws Exception { + super.open(parameters); + collector = new TimestampedCollector(output); + } + + @Override + public void processElement1(StreamRecord element) throws Exception { + collector.setTimestamp(element.getTimestamp()); + userFunction.flatMap1(element.getValue(), collector); } @Override - public void processElement2(IN2 element) throws Exception { - userFunction.flatMap2(element, output); + public void processElement2(StreamRecord element) throws Exception { + collector.setTimestamp(element.getTimestamp()); + userFunction.flatMap2(element.getValue(), collector); + } + + @Override + public void processWatermark1(Watermark mark) throws Exception { + input1Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); + } + } + + @Override + public void processWatermark2(Watermark mark) throws Exception { + input2Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); + } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java index 3dc509a027f9f..b46a929f8fab4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java @@ -22,6 +22,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.functions.co.CoReduceFunction; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class CoStreamGroupedReduce extends CoStreamReduce { @@ -44,30 +45,33 @@ public CoStreamGroupedReduce(CoReduceFunction coReducer, } @Override - public void processElement1(IN1 element) throws Exception { + public void processElement1(StreamRecord elementRecord) throws Exception { + IN1 element = elementRecord.getValue(); Object key = keySelector1.getKey(element); currentValue1 = values1.get(key); if (currentValue1 != null) { reduced1 = userFunction.reduce1(currentValue1, element); values1.put(key, reduced1); - output.collect(userFunction.map1(reduced1)); + output.collect(elementRecord.replace(userFunction.map1(reduced1))); } else { values1.put(key, element); - output.collect(userFunction.map1(element)); + output.collect(elementRecord.replace(userFunction.map1(element))); } } @Override - public void processElement2(IN2 element) throws Exception { + public void processElement2(StreamRecord elementRecord) throws Exception { + IN2 element = elementRecord.getValue(); + Object key = keySelector2.getKey(element); currentValue2 = values2.get(key); if (currentValue2 != null) { reduced2 = userFunction.reduce2(currentValue2, element); values2.put(key, reduced2); - output.collect(userFunction.map2(reduced2)); + output.collect(elementRecord.replace(userFunction.map2(reduced2))); } else { values2.put(key, element); - output.collect(userFunction.map2(element)); + output.collect(elementRecord.replace(userFunction.map2(element))); } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java index a8e57e35ba8d0..8d7c7c4256e20 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java @@ -20,6 +20,8 @@ import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class CoStreamMap extends AbstractUdfStreamOperator> @@ -27,19 +29,43 @@ public class CoStreamMap private static final long serialVersionUID = 1L; + // We keep track of watermarks from both inputs, the combined input is the minimum + // Once the minimum advances we emit a new watermark for downstream operators + private long combinedWatermark = Long.MIN_VALUE; + private long input1Watermark = Long.MAX_VALUE; + private long input2Watermark = Long.MAX_VALUE; + public CoStreamMap(CoMapFunction mapper) { super(mapper); } @Override - public void processElement1(IN1 element) throws Exception { - output.collect(userFunction.map1(element)); + public void processElement1(StreamRecord element) throws Exception { + output.collect(element.replace(userFunction.map1(element.getValue()))); + } + @Override + public void processElement2(StreamRecord element) throws Exception { + output.collect(element.replace(userFunction.map2(element.getValue()))); } @Override - public void processElement2(IN2 element) throws Exception { - output.collect(userFunction.map2(element)); + public void processWatermark1(Watermark mark) throws Exception { + input1Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); + } + } + @Override + public void processWatermark2(Watermark mark) throws Exception { + input2Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); + } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java index 7157b1df6b09b..8609eabda5280 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java @@ -20,6 +20,8 @@ import org.apache.flink.streaming.api.functions.co.CoReduceFunction; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; public class CoStreamReduce extends AbstractUdfStreamOperator> @@ -30,6 +32,12 @@ public class CoStreamReduce protected IN1 currentValue1 = null; protected IN2 currentValue2 = null; + // We keep track of watermarks from both inputs, the combined input is the minimum + // Once the minimum advances we emit a new watermark for downstream operators + private long combinedWatermark = Long.MIN_VALUE; + private long input1Watermark = Long.MAX_VALUE; + private long input2Watermark = Long.MAX_VALUE; + public CoStreamReduce(CoReduceFunction coReducer) { super(coReducer); currentValue1 = null; @@ -37,23 +45,42 @@ public CoStreamReduce(CoReduceFunction coReducer) { } @Override - public void processElement1(IN1 element) throws Exception { + public void processElement1(StreamRecord element) throws Exception { if (currentValue1 != null) { - currentValue1 = userFunction.reduce1(currentValue1, element); + currentValue1 = userFunction.reduce1(currentValue1, element.getValue()); } else { - currentValue1 = element; + currentValue1 = element.getValue(); } - output.collect(userFunction.map1(currentValue1)); + output.collect(element.replace(userFunction.map1(currentValue1))); } @Override - public void processElement2(IN2 element) throws Exception { + public void processElement2(StreamRecord element) throws Exception { if (currentValue2 != null) { - currentValue2 = userFunction.reduce2(currentValue2, element); + currentValue2 = userFunction.reduce2(currentValue2, element.getValue()); } else { - currentValue2 = element; + currentValue2 = element.getValue(); + } + output.collect(element.replace(userFunction.map2(currentValue2))); + } + + @Override + public void processWatermark1(Watermark mark) throws Exception { + input1Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); } - output.collect(userFunction.map2(currentValue2)); } + @Override + public void processWatermark2(Watermark mark) throws Exception { + input2Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); + } + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java index e7b069e6cb464..40d0a89ccc4b6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java @@ -24,8 +24,10 @@ import org.apache.commons.math.util.MathUtils; import org.apache.flink.streaming.api.functions.co.CoWindowFunction; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; +import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.state.CircularFifoList; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -47,6 +49,12 @@ public class CoStreamWindow protected long startTime; protected long nextRecordTime; + // We keep track of watermarks from both inputs, the combined input is the minimum + // Once the minimum advances we emit a new watermark for downstream operators + private long combinedWatermark = Long.MIN_VALUE; + private long input1Watermark = Long.MAX_VALUE; + private long input2Watermark = Long.MAX_VALUE; + public CoStreamWindow(CoWindowFunction coWindowFunction, long windowSize, long slideInterval, TimestampWrapper timeStamp1, TimestampWrapper timeStamp2) { super(coWindowFunction); @@ -62,13 +70,13 @@ public CoStreamWindow(CoWindowFunction coWindowFunction, long win } @Override - public void processElement1(IN1 element) throws Exception { - window.addToBuffer1(element); + public void processElement1(StreamRecord element) throws Exception { + window.addToBuffer1(element.getValue()); } @Override - public void processElement2(IN2 element) throws Exception { - window.addToBuffer2(element); + public void processElement2(StreamRecord element) throws Exception { + window.addToBuffer2(element.getValue()); } @SuppressWarnings("unchecked") @@ -86,8 +94,30 @@ protected void callUserFunction() throws Exception { second.add(element); } + TimestampedCollector timestampedCollector = new TimestampedCollector(output); + timestampedCollector.setTimestamp(System.currentTimeMillis()); if (!window.circularList1.isEmpty() || !window.circularList2.isEmpty()) { - userFunction.coWindow(first, second, output); + userFunction.coWindow(first, second, timestampedCollector); + } + } + + @Override + public void processWatermark1(Watermark mark) throws Exception { + input1Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); + } + } + + @Override + public void processWatermark2(Watermark mark) throws Exception { + input2Watermark = mark.getTimestamp(); + long newMin = Math.min(input1Watermark, input2Watermark); + if (newMin > combinedWatermark && input1Watermark != Long.MAX_VALUE && input2Watermark != Long.MAX_VALUE) { + combinedWatermark = newMin; + output.emitWatermark(new Watermark(combinedWatermark)); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java index 0cdafd95e55aa..0de16b23f169c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java @@ -20,6 +20,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.windowing.policy.CentralActiveTrigger; import org.apache.flink.streaming.api.windowing.policy.CloneableEvictionPolicy; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,9 +52,11 @@ protected StreamDiscretizer makeNewGroup(Object key) throws Exception { } @Override - public void processElement(IN element) throws Exception { - last = element; - Object key = keySelector.getKey(element); + public void processElement(StreamRecord element) throws Exception { + +// last = copy(element); + last = element.getValue(); + Object key = keySelector.getKey(element.getValue()); synchronized (groupedDiscretizers) { StreamDiscretizer groupDiscretizer = groupedDiscretizers.get(key); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java index 64e8b040f970d..e3cab5caf0cab 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.api.windowing.policy.CloneableEvictionPolicy; import org.apache.flink.streaming.api.windowing.policy.CloneableTriggerPolicy; import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * This operator represents the grouped discretization step of a window @@ -67,9 +68,10 @@ public void close() throws Exception { } @Override - public void processElement(IN element) throws Exception { + public void processElement(StreamRecord element) throws Exception { - Object key = keySelector.getKey(element); + + Object key = keySelector.getKey(element.getValue()); StreamDiscretizer groupDiscretizer = groupedDiscretizers.get(key); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java index c6b2499f12816..c74b96e4921fb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.WindowEvent; import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * This operator flattens the results of the window transformations by @@ -51,9 +52,9 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE } @Override - public void processElement(WindowEvent event) throws Exception { - if (event.getElement() != null) { - Object key = keySelector.getKey(event.getElement()); + public void processElement(StreamRecord> event) throws Exception { + if (event.getValue().getElement() != null) { + Object key = keySelector.getKey(event.getValue().getElement()); WindowBuffer currentWindow = windowMap.get(key); if (currentWindow == null) { @@ -61,7 +62,7 @@ public void processElement(WindowEvent event) throws Exception { windowMap.put(key, currentWindow); } - handleWindowEvent(event, currentWindow); + handleWindowEvent(event.getValue(), currentWindow); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java index 4ab31cb960a5a..df84b623403c9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java @@ -19,6 +19,7 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.WindowEvent; import org.apache.flink.streaming.api.windowing.policy.ActiveEvictionPolicy; @@ -26,6 +27,7 @@ import org.apache.flink.streaming.api.windowing.policy.ActiveTriggerPolicy; import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy; import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * This operator represents the discretization step of a window transformation. @@ -67,7 +69,7 @@ public EvictionPolicy getEviction() { } @Override - public void processElement(IN element) throws Exception { + public void processElement(StreamRecord element) throws Exception { processRealElement(element); } @@ -80,13 +82,13 @@ public void processElement(IN element) throws Exception { * a real input element * @throws Exception */ - protected synchronized void processRealElement(IN input) throws Exception { + protected synchronized void processRealElement(StreamRecord input) throws Exception { // Setting the input element in order to avoid NullFieldException when triggering on fake element - windowEvent.setElement(input); + windowEvent.setElement(input.getValue()); if (isActiveTrigger) { ActiveTriggerPolicy trigger = (ActiveTriggerPolicy) triggerPolicy; - Object[] result = trigger.preNotifyTrigger(input); + Object[] result = trigger.preNotifyTrigger(input.getValue()); for (Object in : result) { triggerOnFakeElement(in); } @@ -94,14 +96,14 @@ protected synchronized void processRealElement(IN input) throws Exception { boolean isTriggered = false; - if (triggerPolicy.notifyTrigger(input)) { + if (triggerPolicy.notifyTrigger(input.getValue())) { emitWindow(); isTriggered = true; } - evict(input, isTriggered); + evict(input.getValue(), isTriggered); - output.collect(windowEvent.setElement(input)); + output.collect(input.replace(windowEvent.setElement(input.getValue()))); bufferSize++; } @@ -109,7 +111,7 @@ protected synchronized void processRealElement(IN input) throws Exception { /** * This method triggers on an arrived fake element The method is * synchronized to ensure that it cannot interleave with - * {@link StreamDiscretizer#processRealElement(Object)} + * {@link StreamDiscretizer#processRealElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord)} * * @param input * a fake input element @@ -130,7 +132,7 @@ protected synchronized void triggerOnFakeElement(Object input) { * if not empty */ protected void emitWindow() { - output.collect(windowEvent.setTrigger()); + output.collect(new StreamRecord(windowEvent.setTrigger())); } private void activeEvict(Object input) { @@ -142,7 +144,7 @@ private void activeEvict(Object input) { } if (numToEvict > 0) { - output.collect(windowEvent.setEviction(numToEvict)); + output.collect(new StreamRecord(windowEvent.setEviction(numToEvict))); bufferSize -= numToEvict; bufferSize = bufferSize >= 0 ? bufferSize : 0; } @@ -152,7 +154,7 @@ private void evict(IN input, boolean isTriggered) { int numToEvict = evictionPolicy.notifyEviction(input, isTriggered, bufferSize); if (numToEvict > 0) { - output.collect(windowEvent.setEviction(numToEvict)); + output.collect(new StreamRecord(windowEvent.setEviction(numToEvict))); bufferSize -= numToEvict; bufferSize = bufferSize >= 0 ? bufferSize : 0; } @@ -220,4 +222,9 @@ public String toString() { return "Discretizer(Trigger: " + triggerPolicy.toString() + ", Eviction: " + evictionPolicy.toString() + ")"; } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java index 074ff4be80cee..c057f9183a2f0 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java @@ -19,9 +19,11 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.WindowEvent; import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * This operator manages the window buffers attached to the discretizers. @@ -41,8 +43,8 @@ public StreamWindowBuffer(WindowBuffer buffer) { } @Override - public void processElement(WindowEvent windowEvent) throws Exception { - handleWindowEvent(windowEvent); + public void processElement(StreamRecord> windowEvent) throws Exception { + handleWindowEvent(windowEvent.getValue()); } protected void handleWindowEvent(WindowEvent windowEvent, WindowBuffer buffer) @@ -60,4 +62,8 @@ private void handleWindowEvent(WindowEvent windowEvent) throws Exception { handleWindowEvent(windowEvent, buffer); } + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java index 159b6f8065a4c..fa7696ad83d12 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java @@ -19,7 +19,9 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * This operator flattens the results of the window transformations by @@ -36,9 +38,14 @@ public WindowFlattener() { } @Override - public void processElement(StreamWindow window) throws Exception { - for (T element : window) { - output.collect(element); + public void processElement(StreamRecord> window) throws Exception { + for (T element : window.getValue()) { + output.collect(new StreamRecord(element)); } } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java index 93a92f4bc88bd..9ed5e8275a4e4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java @@ -22,7 +22,9 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * This operator merges together the different partitions of the @@ -44,7 +46,8 @@ public WindowMerger() { @Override @SuppressWarnings("unchecked") - public void processElement(StreamWindow nextWindow) throws Exception { + public void processElement(StreamRecord> nextWindowRecord) throws Exception { + StreamWindow nextWindow = nextWindowRecord.getValue(); StreamWindow current = windows.get(nextWindow.windowID); @@ -55,10 +58,16 @@ public void processElement(StreamWindow nextWindow) throws Exception { } if (current.numberOfParts == 1) { - output.collect(current); + nextWindowRecord.replace(current); + output.collect(nextWindowRecord); windows.remove(nextWindow.windowID); } else { windows.put(nextWindow.windowID, current); } } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java index 6b10c16b8cc23..9f31fa0cca78e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java @@ -20,7 +20,9 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * This operator applies either split or key partitioning depending on the @@ -48,23 +50,30 @@ public WindowPartitioner(int numberOfSplits) { } @Override - public void processElement(StreamWindow currentWindow) throws Exception { + public void processElement(StreamRecord> currentWindow) throws Exception { if (keySelector == null) { if (numberOfSplits <= 1) { output.collect(currentWindow); } else { - for (StreamWindow window : StreamWindow.split(currentWindow, numberOfSplits)) { - output.collect(window); + StreamWindow unpackedWindow = currentWindow.getValue(); + for (StreamWindow window : StreamWindow.split(unpackedWindow, numberOfSplits)) { + currentWindow.replace(window); + output.collect(currentWindow); } } } else { for (StreamWindow window : StreamWindow - .partitionBy(currentWindow, keySelector, true)) { - output.collect(window); + .partitionBy(currentWindow.getValue(), keySelector, true)) { + output.collect(new StreamRecord>(window)); } } } + + @Override + public void processWatermark(Watermark mark) throws Exception { + output.emitWatermark(mark); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java index bfc160f6fae8b..372cb1067858c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java @@ -81,7 +81,7 @@ public S value() throws IOException{ return defaultState; } } catch (Exception e) { - throw new RuntimeException("User-defined key selector threw an exception."); + throw new RuntimeException("User-defined key selector threw an exception.", e); } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java new file mode 100644 index 0000000000000..1d88fe2d2bd3a --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java @@ -0,0 +1,79 @@ +/** + * 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.watermark; + +/** + * A Watermark tells operators that receive it that no elements with a timestamp older or equal + * to the watermark timestamp should arrive at the operator. Watermarks are emitted at the + * sources and propagate through the operators of the topology. Operators must themselves emit + * watermarks to downstream operators using + * {@link org.apache.flink.streaming.api.operators.Output#emitWatermark(Watermark)}. Operators that + * do not internally buffer elements can always forward the watermark that they receive. Operators + * that buffer elements, such as window operators, must forward a watermark after emission of + * elements that is triggered by the arriving watermark. + * + *

+ * In some cases a watermark is only a heuristic and operators should be able to deal with + * late elements. They can either discard those or update the result and emit updates/retractions + * to downstream operations. + * + */ +public class Watermark { + + private long timestamp; + + /** + * Creates a new watermark with the given timestamp. + */ + public Watermark(long timestamp) { + this.timestamp = timestamp; + } + + /** + * Returns the timestamp associated with this {@link Watermark} in milliseconds. + */ + public long getTimestamp() { + return timestamp; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Watermark watermark = (Watermark) o; + + return timestamp == watermark.timestamp; + } + + @Override + public int hashCode() { + return (int) (timestamp ^ (timestamp >>> 32)); + } + + @Override + public String toString() { + return "Watermark{" + + "timestamp=" + timestamp + + '}'; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java index 371e20d71124d..33fb29d4c659f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java @@ -21,6 +21,7 @@ import java.util.NoSuchElementException; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; /** @@ -36,11 +37,11 @@ public BasicWindowBuffer() { this.buffer = new LinkedList(); } - public void emitWindow(Collector> collector) { + public void emitWindow(Collector>> collector) { if (emitEmpty || !buffer.isEmpty()) { StreamWindow currentWindow = createEmptyWindow(); currentWindow.addAll(buffer); - collector.collect(currentWindow); + collector.collect(new StreamRecord>(currentWindow)); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java index 1f7c83e3f35ab..195a96619dfad 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; public class JumpingCountGroupedPreReducer extends TumblingGroupedPreReducer { @@ -37,7 +38,7 @@ public JumpingCountGroupedPreReducer(ReduceFunction reducer, KeySelector> collector) { + public void emitWindow(Collector>> collector) { super.emitWindow(collector); skipped = 0; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java index 355d0cece1a12..17fe408d2ef06 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; /** @@ -39,7 +40,7 @@ public JumpingCountPreReducer(ReduceFunction reducer, TypeSerializer seria } @Override - public void emitWindow(Collector> collector) { + public void emitWindow(Collector>> collector) { super.emitWindow(collector); skipped = 0; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java index f2386a8d63729..a92fc98a12606 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java @@ -22,6 +22,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; public class JumpingTimeGroupedPreReducer extends TumblingGroupedPreReducer { @@ -42,7 +43,7 @@ public JumpingTimeGroupedPreReducer(ReduceFunction reducer, KeySelector } @Override - public void emitWindow(Collector> collector) { + public void emitWindow(Collector>> collector) { super.emitWindow(collector); windowStartTime += slideSize; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java index 98c264d461558..1a47bc8401fe9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; /** @@ -44,7 +45,7 @@ public JumpingTimePreReducer(ReduceFunction reducer, TypeSerializer serial } @Override - public void emitWindow(Collector> collector) { + public void emitWindow(Collector>> collector) { super.emitWindow(collector); windowStartTime += slideSize; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java index 3a2decf1171fc..e2c46a37baa24 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; /** @@ -49,12 +50,12 @@ public SlidingPreReducer(ReduceFunction reducer, TypeSerializer serializer this.serializer = serializer; } - public void emitWindow(Collector> collector) { + public void emitWindow(Collector>> collector) { StreamWindow currentWindow = createEmptyWindow(); try { if (addFinalAggregate(currentWindow) || emitEmpty) { - collector.collect(currentWindow); + collector.collect(new StreamRecord>(currentWindow)); } afterEmit(); } catch (Exception e) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.java index d2f6234b265cf..37d3aae825481 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; /** @@ -56,14 +57,14 @@ public TumblingGroupedPreReducer(ReduceFunction reducer, KeySelector ke this.evict = evict; } - public void emitWindow(Collector> collector) { + public void emitWindow(Collector>> collector) { if (!reducedValues.isEmpty()) { StreamWindow currentWindow = createEmptyWindow(); currentWindow.addAll(reducedValues.values()); - collector.collect(currentWindow); + collector.collect(new StreamRecord>(currentWindow)); } else if (emitEmpty) { - collector.collect(createEmptyWindow()); + collector.collect(new StreamRecord>(createEmptyWindow())); } if (evict) { reducedValues.clear(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java index f396e414295c1..3a10be7d86730 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; /** @@ -48,13 +49,13 @@ private TumblingPreReducer(ReduceFunction reducer, TypeSerializer serializ this.evict = evict; } - public void emitWindow(Collector> collector) { + public void emitWindow(Collector>> collector) { if (reduced != null) { StreamWindow currentWindow = createEmptyWindow(); currentWindow.add(reduced); - collector.collect(currentWindow); + collector.collect(new StreamRecord>(currentWindow)); } else if (emitEmpty) { - collector.collect(createEmptyWindow()); + collector.collect(new StreamRecord>(createEmptyWindow())); } if (evict) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.java index 5c5ea52b887bc..6e87d0b81b479 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.java @@ -20,6 +20,7 @@ import java.io.Serializable; import org.apache.flink.streaming.api.windowing.StreamWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; /** @@ -39,7 +40,7 @@ public abstract class WindowBuffer implements Serializable, Cloneable { public abstract void evict(int n); - public abstract void emitWindow(Collector> collector); + public abstract void emitWindow(Collector>> collector); public abstract WindowBuffer clone(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java index 8f8325f95d47e..40e84fcdf69dd 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java @@ -28,31 +28,30 @@ import org.apache.flink.runtime.io.network.api.reader.AbstractReader; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.streaming.runtime.tasks.StreamingSuperstep; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Class encapsulating the functionality that is necessary to sync inputs on - * superstep barriers. Once a barrier is received from an input channel, whe - * should not process further buffers from that channel until we received the - * barrier from all other channels as well. To avoid back-pressuring the + * The barrier buffer is responsible for implementing the blocking behaviour described + * here: {@link CheckpointBarrier}. + * + *

+ * To avoid back-pressuring the * readers, we buffer up the new data received from the blocked channels until * the blocks are released. - * */ public class BarrierBuffer { private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class); - private Queue nonprocessed = new LinkedList(); - private Queue blockedNonprocessed = new LinkedList(); + private Queue nonProcessed = new LinkedList(); + private Queue blockedNonProcessed = new LinkedList(); private Set blockedChannels = new HashSet(); private int totalNumberOfInputChannels; - private StreamingSuperstep currentSuperstep; - private boolean superstepStarted; + private CheckpointBarrier currentBarrier; private AbstractReader reader; @@ -65,6 +64,8 @@ public class BarrierBuffer { private BufferOrEvent endOfStreamEvent = null; + private long lastCheckpointId = Long.MIN_VALUE; + public BarrierBuffer(InputGate inputGate, AbstractReader reader) { this.inputGate = inputGate; totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); @@ -79,32 +80,18 @@ public BarrierBuffer(InputGate inputGate, AbstractReader reader) { } /** - * Starts the next superstep in the buffer - * - * @param superstep - * The next superstep - */ - protected void startSuperstep(StreamingSuperstep superstep) { - this.currentSuperstep = superstep; - this.superstepStarted = true; - if (LOG.isDebugEnabled()) { - LOG.debug("Superstep started with id: " + superstep.getId()); - } - } - - /** - * Get then next non-blocked non-processed BufferOrEvent. Returns null if - * not available. + * Get then next non-blocked non-processed {@link BufferOrEvent}. Returns null if + * none available. * * @throws IOException */ - protected BufferOrEvent getNonProcessed() throws IOException { - SpillingBufferOrEvent nextNonprocessed; + private BufferOrEvent getNonProcessed() throws IOException { + SpillingBufferOrEvent nextNonProcessed; - while ((nextNonprocessed = nonprocessed.poll()) != null) { - BufferOrEvent boe = nextNonprocessed.getBufferOrEvent(); + while ((nextNonProcessed = nonProcessed.poll()) != null) { + BufferOrEvent boe = nextNonProcessed.getBufferOrEvent(); if (isBlocked(boe.getChannelIndex())) { - blockedNonprocessed.add(new SpillingBufferOrEvent(boe, bufferSpiller, spillReader)); + blockedNonProcessed.add(new SpillingBufferOrEvent(boe, bufferSpiller, spillReader)); } else { return boe; } @@ -114,25 +101,24 @@ protected BufferOrEvent getNonProcessed() throws IOException { } /** - * Checks whether a given channel index is blocked for this inputgate + * Checks whether the channel with the given index is blocked. * - * @param channelIndex - * The channel index to check + * @param channelIndex The channel index to check */ - protected boolean isBlocked(int channelIndex) { + private boolean isBlocked(int channelIndex) { return blockedChannels.contains(channelIndex); } /** - * Checks whether all channels are blocked meaning that barriers are + * Checks whether all channels are blocked meaning that barriers have been * received from all channels */ - protected boolean isAllBlocked() { + private boolean isAllBlocked() { return blockedChannels.size() == totalNumberOfInputChannels; } /** - * Returns the next non-blocked BufferOrEvent. This is a blocking operator. + * Returns the next non-blocked {@link BufferOrEvent}. This is a blocking operator. */ public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException { // If there are non-processed buffers from the previously blocked ones, @@ -141,7 +127,7 @@ public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedExceptio if (bufferOrEvent != null) { return bufferOrEvent; - } else if (blockedNonprocessed.isEmpty() && inputFinished) { + } else if (blockedNonProcessed.isEmpty() && inputFinished) { return endOfStreamEvent; } else { // If no non-processed, get new from input @@ -162,7 +148,7 @@ public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedExceptio } else { if (isBlocked(bufferOrEvent.getChannelIndex())) { // If channel blocked we just store it - blockedNonprocessed.add(new SpillingBufferOrEvent(bufferOrEvent, + blockedNonProcessed.add(new SpillingBufferOrEvent(bufferOrEvent, bufferSpiller, spillReader)); } else { return bufferOrEvent; @@ -182,7 +168,7 @@ public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedExceptio * @param channelIndex * The channel index to block. */ - protected void blockChannel(int channelIndex) { + private void blockChannel(int channelIndex) { if (!blockedChannels.contains(channelIndex)) { blockedChannels.add(channelIndex); if (LOG.isDebugEnabled()) { @@ -199,16 +185,14 @@ protected void blockChannel(int channelIndex) { /** * Releases the blocks on all channels. - * - * @throws IOException */ - protected void releaseBlocks() { - if (!nonprocessed.isEmpty()) { + private void releaseBlocks() { + if (!nonProcessed.isEmpty()) { // sanity check throw new RuntimeException("Error in barrier buffer logic"); } - nonprocessed = blockedNonprocessed; - blockedNonprocessed = new LinkedList(); + nonProcessed = blockedNonProcessed; + blockedNonProcessed = new LinkedList(); try { spillReader.setSpillFile(bufferSpiller.getSpillFile()); @@ -218,7 +202,7 @@ protected void releaseBlocks() { } blockedChannels.clear(); - superstepStarted = false; + currentBarrier = null; if (LOG.isDebugEnabled()) { LOG.debug("All barriers received, blocks released"); } @@ -228,28 +212,46 @@ protected void releaseBlocks() { * Method that is executed once the barrier has been received from all * channels. */ - protected void actOnAllBlocked() { + private void actOnAllBlocked() { if (LOG.isDebugEnabled()) { LOG.debug("Publishing barrier to the vertex"); } - if (currentSuperstep != null && !inputFinished) { - reader.publish(currentSuperstep); + if (currentBarrier != null && !inputFinished) { + reader.publish(currentBarrier); + lastCheckpointId = currentBarrier.getId(); } releaseBlocks(); } /** - * Processes a streaming superstep event + * Processes one {@link org.apache.flink.streaming.runtime.tasks.CheckpointBarrier} * - * @param bufferOrEvent - * The BufferOrEvent containing the event + * @param bufferOrEvent The {@link BufferOrEvent} containing the checkpoint barrier */ - public void processSuperstep(BufferOrEvent bufferOrEvent) { - StreamingSuperstep superstep = (StreamingSuperstep) bufferOrEvent.getEvent(); - if (!superstepStarted) { - startSuperstep(superstep); + public void processBarrier(BufferOrEvent bufferOrEvent) { + CheckpointBarrier receivedBarrier = (CheckpointBarrier) bufferOrEvent.getEvent(); + + if (receivedBarrier.getId() < lastCheckpointId) { + // a barrier from an old checkpoint, ignore these + return; + } + + if (currentBarrier == null) { + this.currentBarrier = receivedBarrier; + if (LOG.isDebugEnabled()) { + LOG.debug("Checkpoint barrier received start waiting for checkpoint: {}", receivedBarrier); + } + } else if (receivedBarrier.getId() > currentBarrier.getId()) { + // we have a barrier from a more recent checkpoint, free all locks and start with + // this newer checkpoint + if (LOG.isDebugEnabled()) { + LOG.debug("Checkpoint barrier received while waiting on checkpoint {}. Restarting waiting with checkpoint {}: ", currentBarrier, receivedBarrier); + } + releaseBlocks(); + currentBarrier = receivedBarrier; + } blockChannel(bufferOrEvent.getChannelIndex()); } @@ -269,11 +271,11 @@ public void cleanup() throws IOException { } public String toString() { - return nonprocessed.toString() + blockedNonprocessed.toString(); + return nonProcessed.toString() + blockedNonProcessed.toString(); } public boolean isEmpty() { - return nonprocessed.isEmpty() && blockedNonprocessed.isEmpty(); + return nonProcessed.isEmpty() && blockedNonProcessed.isEmpty(); } -} \ No newline at end of file +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java index 247fe25abaa1d..9bf4eb488d908 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java @@ -1,41 +1,41 @@ /* - * 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 + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and * limitations under the License. - */ - -package org.apache.flink.streaming.runtime.io; - + */ + +package org.apache.flink.streaming.runtime.io; + import java.util.concurrent.BlockingQueue; import org.apache.flink.runtime.iterative.concurrent.Broker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; - -@SuppressWarnings("rawtypes") -public class BlockingQueueBroker extends Broker> { - /** - * Singleton instance - */ - private static final BlockingQueueBroker INSTANCE = new BlockingQueueBroker(); - - private BlockingQueueBroker() { - } - - /** - * retrieve singleton instance - */ - public static Broker> instance() { - return INSTANCE; - } -} + +@SuppressWarnings("rawtypes") +public class BlockingQueueBroker extends Broker> { + /** + * Singleton instance + */ + private static final BlockingQueueBroker INSTANCE = new BlockingQueueBroker(); + + private BlockingQueueBroker() { + } + + /** + * retrieve singleton instance + */ + public static Broker> instance() { + return INSTANCE; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CoReaderIterator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CoReaderIterator.java deleted file mode 100644 index 4358810835612..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CoReaderIterator.java +++ /dev/null @@ -1,57 +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.runtime.io; - -import java.io.IOException; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.plugable.DeserializationDelegate; -import org.apache.flink.runtime.plugable.ReusingDeserializationDelegate; - -/** - * A CoReaderIterator wraps a {@link CoRecordReader} producing records of two - * input types. - */ -public class CoReaderIterator { - - private final CoRecordReader, DeserializationDelegate> reader; // the - // source - - protected final ReusingDeserializationDelegate delegate1; - protected final ReusingDeserializationDelegate delegate2; - - public CoReaderIterator( - CoRecordReader, DeserializationDelegate> reader, - TypeSerializer serializer1, TypeSerializer serializer2) { - this.reader = reader; - this.delegate1 = new ReusingDeserializationDelegate(serializer1); - this.delegate2 = new ReusingDeserializationDelegate(serializer2); - } - - public int next(T1 target1, T2 target2) throws IOException { - this.delegate1.setInstance(target1); - this.delegate2.setInstance(target2); - - try { - return this.reader.getNextRecord(this.delegate1, this.delegate2); - - } catch (InterruptedException e) { - throw new IOException("Reader interrupted.", e); - } - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CoRecordReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CoRecordReader.java deleted file mode 100644 index a7139b677b04c..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CoRecordReader.java +++ /dev/null @@ -1,300 +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.runtime.io; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.concurrent.LinkedBlockingDeque; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.io.network.api.reader.AbstractReader; -import org.apache.flink.runtime.io.network.api.reader.MutableRecordReader; -import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer; -import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; -import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.StreamingSuperstep; - -/** - * A CoRecordReader wraps {@link MutableRecordReader}s of two different input - * types to read records effectively. - */ -@SuppressWarnings("rawtypes") -public class CoRecordReader extends - AbstractReader implements EventListener, StreamingReader { - - private final InputGate bufferReader1; - - private final InputGate bufferReader2; - - private final LinkedBlockingDeque availableRecordReaders = new LinkedBlockingDeque(); - - private LinkedList processed = new LinkedList(); - - private AdaptiveSpanningRecordDeserializer[] reader1RecordDeserializers; - - private RecordDeserializer reader1currentRecordDeserializer; - - private AdaptiveSpanningRecordDeserializer[] reader2RecordDeserializers; - - private RecordDeserializer reader2currentRecordDeserializer; - - // 0 => none, 1 => reader (T1), 2 => reader (T2) - private int currentReaderIndex; - - private boolean hasRequestedPartitions; - - protected CoBarrierBuffer barrierBuffer1; - protected CoBarrierBuffer barrierBuffer2; - - public CoRecordReader(InputGate inputgate1, InputGate inputgate2) { - super(new UnionInputGate(inputgate1, inputgate2)); - - this.bufferReader1 = inputgate1; - this.bufferReader2 = inputgate2; - - this.reader1RecordDeserializers = new AdaptiveSpanningRecordDeserializer[inputgate1 - .getNumberOfInputChannels()]; - this.reader2RecordDeserializers = new AdaptiveSpanningRecordDeserializer[inputgate2 - .getNumberOfInputChannels()]; - - for (int i = 0; i < reader1RecordDeserializers.length; i++) { - reader1RecordDeserializers[i] = new AdaptiveSpanningRecordDeserializer(); - } - - for (int i = 0; i < reader2RecordDeserializers.length; i++) { - reader2RecordDeserializers[i] = new AdaptiveSpanningRecordDeserializer(); - } - - inputgate1.registerListener(this); - inputgate2.registerListener(this); - - barrierBuffer1 = new CoBarrierBuffer(inputgate1, this); - barrierBuffer2 = new CoBarrierBuffer(inputgate2, this); - - barrierBuffer1.setOtherBarrierBuffer(barrierBuffer2); - barrierBuffer2.setOtherBarrierBuffer(barrierBuffer1); - } - - public void requestPartitionsOnce() throws IOException, InterruptedException { - if (!hasRequestedPartitions) { - bufferReader1.requestPartitions(); - bufferReader2.requestPartitions(); - - hasRequestedPartitions = true; - } - } - - @SuppressWarnings("unchecked") - protected int getNextRecord(T1 target1, T2 target2) throws IOException, InterruptedException { - - requestPartitionsOnce(); - - while (true) { - if (currentReaderIndex == 0) { - if ((bufferReader1.isFinished() && bufferReader2.isFinished())) { - return 0; - } - - currentReaderIndex = getNextReaderIndexBlocking(); - - } - - if (currentReaderIndex == 1) { - while (true) { - if (reader1currentRecordDeserializer != null) { - RecordDeserializer.DeserializationResult result = reader1currentRecordDeserializer - .getNextRecord(target1); - - if (result.isBufferConsumed()) { - reader1currentRecordDeserializer.getCurrentBuffer().recycle(); - reader1currentRecordDeserializer = null; - - currentReaderIndex = 0; - } - - if (result.isFullRecord()) { - return 1; - } - } else { - - final BufferOrEvent boe = barrierBuffer1.getNextNonBlocked(); - - if (boe.isBuffer()) { - reader1currentRecordDeserializer = reader1RecordDeserializers[boe - .getChannelIndex()]; - reader1currentRecordDeserializer.setNextBuffer(boe.getBuffer()); - } else if (boe.getEvent() instanceof StreamingSuperstep) { - barrierBuffer1.processSuperstep(boe); - currentReaderIndex = 0; - - break; - } else if (handleEvent(boe.getEvent())) { - currentReaderIndex = 0; - - break; - } - } - } - } else if (currentReaderIndex == 2) { - while (true) { - if (reader2currentRecordDeserializer != null) { - RecordDeserializer.DeserializationResult result = reader2currentRecordDeserializer - .getNextRecord(target2); - - if (result.isBufferConsumed()) { - reader2currentRecordDeserializer.getCurrentBuffer().recycle(); - reader2currentRecordDeserializer = null; - - currentReaderIndex = 0; - } - - if (result.isFullRecord()) { - return 2; - } - } else { - final BufferOrEvent boe = barrierBuffer2.getNextNonBlocked(); - - if (boe.isBuffer()) { - reader2currentRecordDeserializer = reader2RecordDeserializers[boe - .getChannelIndex()]; - reader2currentRecordDeserializer.setNextBuffer(boe.getBuffer()); - } else if (boe.getEvent() instanceof StreamingSuperstep) { - barrierBuffer2.processSuperstep(boe); - currentReaderIndex = 0; - - break; - } else if (handleEvent(boe.getEvent())) { - currentReaderIndex = 0; - - break; - } - } - } - } else { - throw new IllegalStateException("Bug: unexpected current reader index."); - } - } - } - - protected int getNextReaderIndexBlocking() throws InterruptedException { - - Integer nextIndex = 0; - - while (processed.contains(nextIndex = availableRecordReaders.take())) { - processed.remove(nextIndex); - } - - if (nextIndex == 1) { - if (barrierBuffer1.isAllBlocked()) { - availableRecordReaders.addFirst(1); - processed.add(2); - return 2; - } else { - return 1; - } - } else { - if (barrierBuffer2.isAllBlocked()) { - availableRecordReaders.addFirst(2); - processed.add(1); - return 1; - } else { - return 2; - } - - } - - } - - // ------------------------------------------------------------------------ - // Data availability notifications - // ------------------------------------------------------------------------ - - @Override - public void onEvent(InputGate bufferReader) { - addToAvailable(bufferReader); - } - - protected void addToAvailable(InputGate bufferReader) { - if (bufferReader == bufferReader1) { - availableRecordReaders.add(1); - } else if (bufferReader == bufferReader2) { - availableRecordReaders.add(2); - } - } - - public void clearBuffers() { - for (RecordDeserializer deserializer : reader1RecordDeserializers) { - Buffer buffer = deserializer.getCurrentBuffer(); - if (buffer != null && !buffer.isRecycled()) { - buffer.recycle(); - } - } - for (RecordDeserializer deserializer : reader2RecordDeserializers) { - Buffer buffer = deserializer.getCurrentBuffer(); - if (buffer != null && !buffer.isRecycled()) { - buffer.recycle(); - } - } - } - - @Override - public void setReporter(AccumulatorRegistry.Reporter reporter) { - for (AdaptiveSpanningRecordDeserializer serializer : reader1RecordDeserializers) { - serializer.setReporter(reporter); - } - for (AdaptiveSpanningRecordDeserializer serializer : reader2RecordDeserializers) { - serializer.setReporter(reporter); - } - } - - private class CoBarrierBuffer extends BarrierBuffer { - - private CoBarrierBuffer otherBuffer; - - public CoBarrierBuffer(InputGate inputGate, AbstractReader reader) { - super(inputGate, reader); - } - - public void setOtherBarrierBuffer(CoBarrierBuffer other) { - this.otherBuffer = other; - } - - @Override - protected void actOnAllBlocked() { - if (otherBuffer.isAllBlocked()) { - super.actOnAllBlocked(); - otherBuffer.releaseBlocks(); - } - } - - } - - public void cleanup() throws IOException { - try { - barrierBuffer1.cleanup(); - } finally { - barrierBuffer2.cleanup(); - } - - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java similarity index 63% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java index 2fd4cd0f0518c..2f9d1d693052f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java @@ -15,32 +15,48 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.collector; +package org.apache.flink.streaming.runtime.io; import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; -public class CollectorWrapper implements Output { +import java.util.ArrayList; +import java.util.List; + +public class CollectorWrapper implements Output> { private OutputSelectorWrapper outputSelectorWrapper; + private List> allOutputs; + public CollectorWrapper(OutputSelectorWrapper outputSelectorWrapper) { this.outputSelectorWrapper = outputSelectorWrapper; + allOutputs = new ArrayList>(); } - public void addCollector(Collector output, StreamEdge edge) { + public void addCollector(Collector> output, StreamEdge edge) { outputSelectorWrapper.addCollector(output, edge); + allOutputs.add((Output) output); } @Override - public void collect(OUT record) { - for (Collector output : outputSelectorWrapper.getSelectedOutputs(record)) { + public void collect(StreamRecord record) { + for (Collector> output : outputSelectorWrapper.getSelectedOutputs(record.getValue())) { output.collect(record); } } + @Override + public void emitWatermark(Watermark mark) { + for (Output output : allOutputs) { + output.emitWatermark(mark); + } + } + @Override public void close() { } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/IndexedMutableReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/IndexedMutableReader.java deleted file mode 100644 index 7f2a9c51b4675..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/IndexedMutableReader.java +++ /dev/null @@ -1,37 +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.runtime.io; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; - -public class IndexedMutableReader extends - StreamingMutableRecordReader { - - InputGate reader; - - public IndexedMutableReader(InputGate reader) { - super(reader); - this.reader = reader; - } - - public int getNumberOfInputChannels() { - return reader.getNumberOfInputChannels(); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/IndexedReaderIterator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/IndexedReaderIterator.java deleted file mode 100644 index 2050e27834522..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/IndexedReaderIterator.java +++ /dev/null @@ -1,33 +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.runtime.io; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.util.ReaderIterator; -import org.apache.flink.runtime.plugable.DeserializationDelegate; - -public class IndexedReaderIterator extends ReaderIterator { - - public IndexedReaderIterator( - IndexedMutableReader> reader, - TypeSerializer serializer) { - - super(reader, serializer); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateUtil.java similarity index 70% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateFactory.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateUtil.java index 788325187e037..01e16fbc6fbb4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateFactory.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateUtil.java @@ -17,15 +17,25 @@ package org.apache.flink.streaming.runtime.io; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; -public class InputGateFactory { +/** + * Utility for dealing with input gates. This will either just return + * the single {@link InputGate} that was passed in or create a {@link UnionInputGate} if several + * {@link InputGate input gates} are given. + */ +public class InputGateUtil { - public static InputGate createInputGate(Collection inputGates) { - return createInputGate(inputGates.toArray(new InputGate[inputGates.size()])); + public static InputGate createInputGate(Collection inputGates1, Collection inputGates2) { + List gates = new ArrayList(inputGates1.size() + inputGates2.size()); + gates.addAll(inputGates1); + gates.addAll(inputGates2); + return createInputGate(gates.toArray(new InputGate[gates.size()])); } public static InputGate createInputGate(InputGate[] inputGates) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java new file mode 100644 index 0000000000000..e9cbb7d6abdae --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.io; + +import java.io.IOException; + +import com.google.common.base.Preconditions; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.io.network.api.writer.RecordWriter; +import org.apache.flink.runtime.plugable.SerializationDelegate; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implementation of {@link Output} that sends data using a {@link RecordWriter}. + */ +public class RecordWriterOutput implements Output> { + + private static final Logger LOG = LoggerFactory.getLogger(RecordWriterOutput.class); + + private RecordWriter recordWriter; + private SerializationDelegate serializationDelegate; + + @SuppressWarnings("unchecked") + public RecordWriterOutput( + RecordWriter recordWriter, + TypeSerializer outSerializer, + boolean enableWatermarkMultiplexing) { + Preconditions.checkNotNull(recordWriter); + + this.recordWriter = recordWriter; + + StreamRecordSerializer outRecordSerializer; + if (enableWatermarkMultiplexing) { + outRecordSerializer = new MultiplexingStreamRecordSerializer(outSerializer); + } else { + outRecordSerializer = new StreamRecordSerializer(outSerializer); + } + + if (outSerializer != null) { + serializationDelegate = new SerializationDelegate(outRecordSerializer); + } + } + + @Override + @SuppressWarnings("unchecked") + public void collect(StreamRecord record) { + serializationDelegate.setInstance(record); + + try { + recordWriter.emit(serializationDelegate); + } catch (Exception e) { + if (LOG.isErrorEnabled()) { + LOG.error("Emit failed: {}", e); + } + throw new RuntimeException("Element emission failed.", e); + } + } + + @Override + @SuppressWarnings("unchecked") + public void emitWatermark(Watermark mark) { + serializationDelegate.setInstance(mark); + try { + recordWriter.broadcastEmit(serializationDelegate); + } catch (Exception e) { + if (LOG.isErrorEnabled()) { + LOG.error("Watermark emit failed: {}", e); + } + throw new RuntimeException(e); + } + } + + @Override + public void close() { + if (recordWriter instanceof StreamRecordWriter) { + ((StreamRecordWriter) recordWriter).close(); + } else { + try { + recordWriter.flush(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + + public void clearBuffers() { + recordWriter.clearBuffers(); + } + + public void broadcastEvent(TaskEvent barrier) throws IOException, InterruptedException { + recordWriter.broadcastEvent(barrier); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java new file mode 100644 index 0000000000000..e665710c70bd6 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.io; + +import java.io.IOException; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.io.network.api.reader.AbstractReader; +import org.apache.flink.runtime.io.network.api.reader.ReaderBase; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult; +import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Input reader for {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}. + * + *

+ * This also keeps track of {@link Watermark} events and forwards them to event subscribers + * once the {@link Watermark} from all inputs advances. + * + * @param The type of the record that can be read with this record reader. + */ +public class StreamInputProcessor extends AbstractReader implements ReaderBase, StreamingReader { + + @SuppressWarnings("unused") + private static final Logger LOG = LoggerFactory.getLogger(StreamInputProcessor.class); + + private final RecordDeserializer[] recordDeserializers; + + private RecordDeserializer currentRecordDeserializer; + + // We need to keep track of the channel from which a buffer came, so that we can + // appropriately map the watermarks to input channels + int currentChannel = -1; + + private boolean isFinished; + + private final BarrierBuffer barrierBuffer; + + private long[] watermarks; + private long lastEmittedWatermark; + + private DeserializationDelegate deserializationDelegate; + + @SuppressWarnings("unchecked") + public StreamInputProcessor(InputGate[] inputGates, TypeSerializer inputSerializer, boolean enableWatermarkMultiplexing) { + super(InputGateUtil.createInputGate(inputGates)); + + barrierBuffer = new BarrierBuffer(inputGate, this); + + StreamRecordSerializer inputRecordSerializer; + if (enableWatermarkMultiplexing) { + inputRecordSerializer = new MultiplexingStreamRecordSerializer(inputSerializer); + } else { + inputRecordSerializer = new StreamRecordSerializer(inputSerializer); + } + this.deserializationDelegate = new NonReusingDeserializationDelegate(inputRecordSerializer); + + // Initialize one deserializer per input channel + this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()]; + for (int i = 0; i < recordDeserializers.length; i++) { + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer(); + } + + watermarks = new long[inputGate.getNumberOfInputChannels()]; + for (int i = 0; i < inputGate.getNumberOfInputChannels(); i++) { + watermarks[i] = Long.MIN_VALUE; + } + lastEmittedWatermark = Long.MIN_VALUE; + } + + @SuppressWarnings("unchecked") + public boolean processInput(OneInputStreamOperator streamOperator) throws Exception { + if (isFinished) { + return false; + } + + while (true) { + if (currentRecordDeserializer != null) { + DeserializationResult result = currentRecordDeserializer.getNextRecord(deserializationDelegate); + + if (result.isBufferConsumed()) { + currentRecordDeserializer.getCurrentBuffer().recycle(); + currentRecordDeserializer = null; + } + + if (result.isFullRecord()) { + Object recordOrWatermark = deserializationDelegate.getInstance(); + + if (recordOrWatermark instanceof Watermark) { + Watermark mark = (Watermark) recordOrWatermark; + long watermarkMillis = mark.getTimestamp(); + if (watermarkMillis > watermarks[currentChannel]) { + watermarks[currentChannel] = watermarkMillis; + long newMinWatermark = Long.MAX_VALUE; + for (long watermark : watermarks) { + if (watermark < newMinWatermark) { + newMinWatermark = watermark; + } + } + if (newMinWatermark > lastEmittedWatermark) { + lastEmittedWatermark = newMinWatermark; + streamOperator.processWatermark(new Watermark(lastEmittedWatermark)); + } + } + continue; + } else { + // now we can do the actual processing + StreamRecord record = (StreamRecord) deserializationDelegate.getInstance(); + StreamingRuntimeContext ctx = streamOperator.getRuntimeContext(); + if (ctx != null) { + ctx.setNextInput(record); + } + streamOperator.processElement(record); + return true; + } + } + } + + final BufferOrEvent bufferOrEvent = barrierBuffer.getNextNonBlocked(); + + if (bufferOrEvent.isBuffer()) { + currentChannel = bufferOrEvent.getChannelIndex(); + currentRecordDeserializer = recordDeserializers[currentChannel]; + currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); + } else { + // Event received + final AbstractEvent event = bufferOrEvent.getEvent(); + + if (event instanceof CheckpointBarrier) { + barrierBuffer.processBarrier(bufferOrEvent); + } else { + if (handleEvent(event)) { + if (inputGate.isFinished()) { + if (!barrierBuffer.isEmpty()) { + throw new RuntimeException("BarrierBuffer should be empty at this point"); + } + isFinished = true; + return false; + } else if (hasReachedEndOfSuperstep()) { + return false; + } // else: More data is coming... + } + } + } + } + } + + @Override + public void setReporter(AccumulatorRegistry.Reporter reporter) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.setReporter(reporter); + } + } + + public void clearBuffers() { + for (RecordDeserializer deserializer : recordDeserializers) { + Buffer buffer = deserializer.getCurrentBuffer(); + if (buffer != null && !buffer.isRecycled()) { + buffer.recycle(); + } + } + } + + public void cleanup() throws IOException { + barrierBuffer.cleanup(); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java index c212346c264f7..abae9a446663b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java @@ -61,6 +61,14 @@ public void emit(T record) throws IOException, InterruptedException { } } + @Override + public void broadcastEmit(T record) throws IOException, InterruptedException { + super.broadcastEmit(record); + if (flushAlways) { + flush(); + } + } + public void close() { try { if (outputFlusher != null) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java new file mode 100644 index 0000000000000..1fe98bbd74f30 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -0,0 +1,268 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.io; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.io.network.api.reader.AbstractReader; +import org.apache.flink.runtime.io.network.api.reader.ReaderBase; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult; +import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; + +/** + * Input reader for {@link org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask}. + * + *

+ * This also keeps track of {@link org.apache.flink.streaming.api.watermark.Watermark} events and forwards them to event subscribers + * once the {@link org.apache.flink.streaming.api.watermark.Watermark} from all inputs advances. + * + * @param The type of the records that arrive on the first input + * @param The type of the records that arrive on the second input + */ +public class StreamTwoInputProcessor extends AbstractReader implements ReaderBase, StreamingReader { + + @SuppressWarnings("unused") + private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputProcessor.class); + + private final RecordDeserializer[] recordDeserializers; + + private RecordDeserializer currentRecordDeserializer; + + // We need to keep track of the channel from which a buffer came, so that we can + // appropriately map the watermarks to input channels + int currentChannel = -1; + + private boolean isFinished; + + private final BarrierBuffer barrierBuffer; + + private long[] watermarks1; + private long lastEmittedWatermark1; + + private long[] watermarks2; + private long lastEmittedWatermark2; + + private int numInputChannels1; + private int numInputChannels2; + + private DeserializationDelegate deserializationDelegate1; + private DeserializationDelegate deserializationDelegate2; + + @SuppressWarnings("unchecked") + public StreamTwoInputProcessor( + Collection inputGates1, + Collection inputGates2, + TypeSerializer inputSerializer1, + TypeSerializer inputSerializer2, + boolean enableWatermarkMultiplexing) { + super(InputGateUtil.createInputGate(inputGates1, inputGates2)); + + barrierBuffer = new BarrierBuffer(inputGate, this); + + StreamRecordSerializer inputRecordSerializer1; + if (enableWatermarkMultiplexing) { + inputRecordSerializer1 = new MultiplexingStreamRecordSerializer(inputSerializer1); + } else { + inputRecordSerializer1 = new StreamRecordSerializer(inputSerializer1); + } + this.deserializationDelegate1 = new NonReusingDeserializationDelegate(inputRecordSerializer1); + + StreamRecordSerializer inputRecordSerializer2; + if (enableWatermarkMultiplexing) { + inputRecordSerializer2 = new MultiplexingStreamRecordSerializer(inputSerializer2); + } else { + inputRecordSerializer2 = new StreamRecordSerializer(inputSerializer2); + } + this.deserializationDelegate2 = new NonReusingDeserializationDelegate(inputRecordSerializer2); + + // Initialize one deserializer per input channel + this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate + .getNumberOfInputChannels()]; + for (int i = 0; i < recordDeserializers.length; i++) { + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer(); + } + + // determine which unioned channels belong to input 1 and which belong to input 2 + numInputChannels1 = 0; + for (InputGate gate: inputGates1) { + numInputChannels1 += gate.getNumberOfInputChannels(); + } + numInputChannels2 = inputGate.getNumberOfInputChannels() - numInputChannels1; + + watermarks1 = new long[numInputChannels1]; + for (int i = 0; i < numInputChannels1; i++) { + watermarks1[i] = Long.MIN_VALUE; + } + lastEmittedWatermark1 = Long.MIN_VALUE; + + watermarks2 = new long[numInputChannels2]; + for (int i = 0; i < numInputChannels2; i++) { + watermarks2[i] = Long.MIN_VALUE; + } + lastEmittedWatermark2 = Long.MIN_VALUE; + } + + @SuppressWarnings("unchecked") + public boolean processInput(TwoInputStreamOperator streamOperator) throws Exception { + if (isFinished) { + return false; + } + + while (true) { + if (currentRecordDeserializer != null) { + DeserializationResult result; + if (currentChannel < numInputChannels1) { + result = currentRecordDeserializer.getNextRecord(deserializationDelegate1); + } else { + result = currentRecordDeserializer.getNextRecord(deserializationDelegate2); + } + + if (result.isBufferConsumed()) { + currentRecordDeserializer.getCurrentBuffer().recycle(); + currentRecordDeserializer = null; + } + + if (result.isFullRecord()) { + if (currentChannel < numInputChannels1) { + Object recordOrWatermark = deserializationDelegate1.getInstance(); + if (recordOrWatermark instanceof Watermark) { + handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel); + continue; + } else { + streamOperator.processElement1((StreamRecord) deserializationDelegate1.getInstance()); + return true; + + } + } else { + Object recordOrWatermark = deserializationDelegate2.getInstance(); + if (recordOrWatermark instanceof Watermark) { + handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel); + continue; + } else { + streamOperator.processElement2((StreamRecord) deserializationDelegate2.getInstance()); + return true; + } + } + } + } + + final BufferOrEvent bufferOrEvent = barrierBuffer.getNextNonBlocked(); + + if (bufferOrEvent.isBuffer()) { + currentChannel = bufferOrEvent.getChannelIndex(); + currentRecordDeserializer = recordDeserializers[currentChannel]; + currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); + + } else { + // Event received + final AbstractEvent event = bufferOrEvent.getEvent(); + + if (event instanceof CheckpointBarrier) { + barrierBuffer.processBarrier(bufferOrEvent); + } else { + if (handleEvent(event)) { + if (inputGate.isFinished()) { + if (!barrierBuffer.isEmpty()) { + throw new RuntimeException("BarrierBuffer should be empty at this point"); + } + isFinished = true; + return false; + } else if (hasReachedEndOfSuperstep()) { + return false; + } // else: More data is coming... + } + } + } + } + } + + private void handleWatermark(TwoInputStreamOperator operator, Watermark mark, int channelIndex) throws Exception { + if (channelIndex < numInputChannels1) { + long watermarkMillis = mark.getTimestamp(); + if (watermarkMillis > watermarks1[channelIndex]) { + watermarks1[channelIndex] = watermarkMillis; + long newMinWatermark = Long.MAX_VALUE; + for (long aWatermarks1 : watermarks1) { + if (aWatermarks1 < newMinWatermark) { + newMinWatermark = aWatermarks1; + } + } + if (newMinWatermark > lastEmittedWatermark1) { + lastEmittedWatermark1 = newMinWatermark; + operator.processWatermark1(new Watermark(lastEmittedWatermark1)); + } + } + } else { + channelIndex = channelIndex - numInputChannels1; + long watermarkMillis = mark.getTimestamp(); + if (watermarkMillis > watermarks2[channelIndex]) { + watermarks2[channelIndex] = watermarkMillis; + long newMinWatermark = Long.MAX_VALUE; + for (long aWatermarks2 : watermarks2) { + if (aWatermarks2 < newMinWatermark) { + newMinWatermark = aWatermarks2; + } + } + if (newMinWatermark > lastEmittedWatermark2) { + lastEmittedWatermark2 = newMinWatermark; + operator.processWatermark2(new Watermark(lastEmittedWatermark2)); + } + } + } + + } + + @Override + public void setReporter(AccumulatorRegistry.Reporter reporter) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.setReporter(reporter); + } + } + + public void clearBuffers() { + for (RecordDeserializer deserializer : recordDeserializers) { + Buffer buffer = deserializer.getCurrentBuffer(); + if (buffer != null && !buffer.isRecycled()) { + buffer.recycle(); + } + } + } + + public void cleanup() throws IOException { + barrierBuffer.cleanup(); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingAbstractRecordReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingAbstractRecordReader.java deleted file mode 100644 index 44f9a868c3a47..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingAbstractRecordReader.java +++ /dev/null @@ -1,144 +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.runtime.io; - -import java.io.IOException; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.runtime.event.task.AbstractEvent; -import org.apache.flink.runtime.io.network.api.reader.AbstractReader; -import org.apache.flink.runtime.io.network.api.reader.ReaderBase; -import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; -import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult; -import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.streaming.runtime.tasks.StreamingSuperstep; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A record-oriented reader. - *

- * This abstract base class is used by both the mutable and immutable record - * readers. - * - * @param - * The type of the record that can be read with this record reader. - */ -public abstract class StreamingAbstractRecordReader extends - AbstractReader implements ReaderBase, StreamingReader { - - @SuppressWarnings("unused") - private static final Logger LOG = LoggerFactory.getLogger(StreamingAbstractRecordReader.class); - - private final RecordDeserializer[] recordDeserializers; - - private RecordDeserializer currentRecordDeserializer; - - private boolean isFinished; - - private final BarrierBuffer barrierBuffer; - - - @SuppressWarnings("unchecked") - protected StreamingAbstractRecordReader(InputGate inputGate) { - super(inputGate); - barrierBuffer = new BarrierBuffer(inputGate, this); - - // Initialize one deserializer per input channel - this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate - .getNumberOfInputChannels()]; - for (int i = 0; i < recordDeserializers.length; i++) { - recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer(); - } - } - - protected boolean getNextRecord(T target) throws IOException, InterruptedException { - if (isFinished) { - return false; - } - - while (true) { - if (currentRecordDeserializer != null) { - DeserializationResult result = currentRecordDeserializer.getNextRecord(target); - - if (result.isBufferConsumed()) { - Buffer currentBuffer = currentRecordDeserializer.getCurrentBuffer(); - currentBuffer.recycle(); - currentRecordDeserializer = null; - } - - if (result.isFullRecord()) { - return true; - } - } - - final BufferOrEvent bufferOrEvent = barrierBuffer.getNextNonBlocked(); - - if (bufferOrEvent.isBuffer()) { - currentRecordDeserializer = recordDeserializers[bufferOrEvent.getChannelIndex()]; - currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); - } else { - // Event received - final AbstractEvent event = bufferOrEvent.getEvent(); - - if (event instanceof StreamingSuperstep) { - barrierBuffer.processSuperstep(bufferOrEvent); - } else { - if (handleEvent(event)) { - if (inputGate.isFinished()) { - if (!barrierBuffer.isEmpty()) { - throw new RuntimeException( - "BarrierBuffer should be empty at this point"); - } - isFinished = true; - return false; - } else if (hasReachedEndOfSuperstep()) { - return false; - } // else: More data is coming... - } - } - } - } - } - - public void clearBuffers() { - for (RecordDeserializer deserializer : recordDeserializers) { - Buffer buffer = deserializer.getCurrentBuffer(); - if (buffer != null && !buffer.isRecycled()) { - buffer.recycle(); - } - } - } - - public void cleanup() throws IOException { - barrierBuffer.cleanup(); - } - - @Override - public void setReporter(AccumulatorRegistry.Reporter reporter) { - for (RecordDeserializer deserializer : recordDeserializers) { - deserializer.setReporter(reporter); - } - } - -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingMutableRecordReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingMutableRecordReader.java deleted file mode 100644 index 1356af5d75185..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingMutableRecordReader.java +++ /dev/null @@ -1,44 +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.runtime.io; - -import java.io.IOException; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.runtime.io.network.api.reader.MutableReader; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; - -public class StreamingMutableRecordReader extends - StreamingAbstractRecordReader implements MutableReader { - - public StreamingMutableRecordReader(InputGate inputGate) { - super(inputGate); - } - - @Override - public boolean next(final T target) throws IOException, InterruptedException { - return getNextRecord(target); - } - - @Override - public void clearBuffers() { - super.clearBuffers(); - } - -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java index 75867cd439d55..6c40c03521c90 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java @@ -44,10 +44,14 @@ public CustomPartitionerWrapper(Partitioner partitioner, KeySelector ke } @Override - public int[] selectChannels(SerializationDelegate> record, - int numberOfOutputChannels) { - - K key = record.getInstance().getKey(keySelector); + public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { + + K key = null; + try { + key = keySelector.getKey(record.getInstance().getValue()); + } catch (Exception e) { + throw new RuntimeException("Could not extract key from " + record.getInstance(), e); + } returnArray[0] = partitioner.partition(key, numberOfOutputChannels); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java index 08c431b5b2295..7026d45203633 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java @@ -42,8 +42,13 @@ public FieldsPartitioner(KeySelector keySelector) { @Override public int[] selectChannels(SerializationDelegate> record, int numberOfOutputChannels) { - returnArray[0] = Math.abs(record.getInstance().getKey(keySelector).hashCode() - % numberOfOutputChannels); + Object key; + try { + key = keySelector.getKey(record.getInstance().getValue()); + } catch (Exception e) { + throw new RuntimeException("Could not extract key from " + record.getInstance().getValue(), e); + } + returnArray[0] = Math.abs(key.hashCode() % numberOfOutputChannels); return returnArray; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java new file mode 100644 index 0000000000000..715f0d28bdecb --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java @@ -0,0 +1,135 @@ +/* + * 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, + * WITHOUStreamRecordWARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.streamrecord; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.streaming.api.watermark.Watermark; + +import java.io.IOException; + +/** + * Serializer for {@link StreamRecord} and {@link Watermark}. This does not behave like a normal + * {@link TypeSerializer}, instead, this is only used at the + * {@link org.apache.flink.streaming.runtime.tasks.StreamTask} level for transmitting + * {@link StreamRecord StreamRecords} and {@link Watermark Watermarks}. This serializer + * can handle both of them, therefore it returns {@link Object} the result has + * to be cast to the correct type. + * + * @param The type of value in the {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord} + */ +public final class MultiplexingStreamRecordSerializer extends StreamRecordSerializer { + + private final long IS_WATERMARK = Long.MIN_VALUE; + + private static final long serialVersionUID = 1L; + + public MultiplexingStreamRecordSerializer(TypeSerializer serializer) { + super(serializer); + if (serializer instanceof MultiplexingStreamRecordSerializer) { + throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer); + } + } + + @Override + @SuppressWarnings("unchecked") + public Object copy(Object from) { + // we can reuse the timestamp since Instant is immutable + if (from instanceof StreamRecord) { + StreamRecord fromRecord = (StreamRecord) from; + return new StreamRecord(typeSerializer.copy(fromRecord.getValue()), fromRecord.getTimestamp()); + } else if (from instanceof Watermark) { + // is immutable + return from; + } else { + throw new RuntimeException("Cannot copy " + from); + } + } + + @Override + @SuppressWarnings("unchecked") + public Object copy(Object from, Object reuse) { + if (from instanceof StreamRecord && reuse instanceof StreamRecord) { + StreamRecord fromRecord = (StreamRecord) from; + StreamRecord reuseRecord = (StreamRecord) reuse; + + reuseRecord.replace(typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue()), fromRecord.getTimestamp()); + return reuse; + } else if (from instanceof Watermark) { + // is immutable + return from; + } else { + throw new RuntimeException("Cannot copy " + from); + } + } + + @Override + @SuppressWarnings("unchecked") + public void serialize(Object value, DataOutputView target) throws IOException { + if (value instanceof StreamRecord) { + StreamRecord record = (StreamRecord) value; + target.writeLong(record.getTimestamp()); + typeSerializer.serialize(record.getValue(), target); + } else if (value instanceof Watermark) { + target.writeLong(IS_WATERMARK); + target.writeLong(((Watermark) value).getTimestamp()); + } + } + + @Override + public Object deserialize(DataInputView source) throws IOException { + long millis = source.readLong(); + + if (millis == IS_WATERMARK) { + return new Watermark(source.readLong()); + } else { + T element = typeSerializer.deserialize(source); + return new StreamRecord(element, millis); + } + } + + @Override + @SuppressWarnings("unchecked") + public Object deserialize(Object reuse, DataInputView source) throws IOException { + long millis = source.readLong(); + + if (millis == IS_WATERMARK) { + return new Watermark(source.readLong()); + + } else { + StreamRecord reuseRecord = (StreamRecord) reuse; + T element = typeSerializer.deserialize(reuseRecord.getValue(), source); + reuseRecord.replace(element, millis); + return reuse; + } + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + long millis = source.readLong(); + target.writeLong(millis); + + if (millis == IS_WATERMARK) { + target.writeLong(source.readLong()); + } else { + typeSerializer.copy(source, target); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java index 66a64b304d9cb..aff030ef67224 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java @@ -17,87 +17,106 @@ package org.apache.flink.streaming.runtime.streamrecord; -import java.io.Serializable; - -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.tuple.Tuple; - /** - * Object for wrapping a tuple or other object with ID used for sending records - * between streaming task in Apache Flink stream processing. + * One value in a data stream. This stores the value and the associated timestamp. */ -public class StreamRecord implements Serializable { - private static final long serialVersionUID = 1L; +public class StreamRecord { - private T streamObject; - public boolean isTuple; + // We store it as Object so that we can reuse a StreamElement for emitting + // elements of a different type while still reusing the timestamp. + private Object value; + private long timestamp; /** - * Creates an empty StreamRecord + * Creates a new {@link StreamRecord} wrapping the given value. The timestamp is set to the + * result of {@code new Instant(0)}. */ - public StreamRecord() { + public StreamRecord(T value) { + this(value, Long.MIN_VALUE + 1); + // be careful to set it to MIN_VALUE + 1, because MIN_VALUE is reserved as the + // special tag to signify that a transmitted element is a Watermark in StreamRecordSerializer } /** - * Gets the wrapped object from the StreamRecord - * - * @return The object wrapped + * Creates a new {@link StreamRecord} wrapping the given value. The timestamp is set to the + * given timestamp. + * + * @param value The value to wrap in this {@link StreamRecord} + * @param timestamp The timestamp in milliseconds */ - public T getObject() { - return streamObject; + public StreamRecord(T value, long timestamp) { + this.value = value; + this.timestamp = timestamp; } /** - * Gets the field of the contained object at the given position. If a tuple - * is wrapped then the getField method is invoked. If the StreamRecord - * contains and object of Basic types only position 0 could be returned. - * - * @param pos - * Position of the field to get. - * @return Returns the object contained in the position. + * Returns the value wrapped in this stream value. */ - public Object getField(int pos) { - if (isTuple) { - return ((Tuple) streamObject).getField(pos); - } else { - if (pos == 0) { - return streamObject; - } else { - throw new IndexOutOfBoundsException(); - } - } + @SuppressWarnings("unchecked") + public T getValue() { + return (T) value; } /** - * Extracts key for the stored object using the keySelector provided. - * - * @param keySelector - * KeySelector for extracting the key - * @return The extracted key + * Returns the timestamp associated with this stream value in milliseconds. */ - public R getKey(KeySelector keySelector) { - try { - return keySelector.getKey(streamObject); - } catch (Exception e) { - throw new RuntimeException("Failed to extract key: " + e.getMessage()); - } + public long getTimestamp() { + return timestamp; + } + + /** + * 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. + * + * @param element Element to set in this stream value + * @return Returns the StreamElement with replaced value + */ + @SuppressWarnings("unchecked") + public StreamRecord replace(X element) { + this.value = element; + return (StreamRecord) this; } /** - * Sets the object stored - * - * @param object - * Object to set - * @return Returns the StreamRecord object + * 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 + * parameter that matches the new value. + * + * @param value The new value to wrap in this {@link StreamRecord} + * @param timestamp The new timestamp in milliseconds + * @return Returns the StreamElement with replaced value */ - public StreamRecord setObject(T object) { - this.streamObject = object; - return this; + @SuppressWarnings("unchecked") + public StreamRecord replace(X value, long timestamp) { + this.timestamp = timestamp; + this.value = value; + return (StreamRecord) this; } @Override - public String toString() { - return streamObject.toString(); + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StreamRecord that = (StreamRecord) o; + + return value.equals(that.value) && timestamp == that.timestamp; } + @Override + public int hashCode() { + int result = value != null ? value.hashCode() : 0; + result = 31 * result + (int) (timestamp ^ (timestamp >>> 32)); + return result; + } + + @Override + public String toString() { + return "Record{" + value + "; " + timestamp + '}'; + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java index 449949919676b..b05eb36830bd8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java @@ -20,26 +20,35 @@ import java.io.IOException; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeinfo.TypeInformation; +import com.google.common.base.Preconditions; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -public final class StreamRecordSerializer extends TypeSerializer> { +/** + * Serializer for {@link StreamRecord}. This version ignores timestamps and only deals with + * the element. + * + *

+ * {@link MultiplexingStreamRecordSerializer} is a version that deals with timestamps and also + * multiplexes {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks} in the same + * stream with {@link StreamRecord StreamRecords}. + * + * @see MultiplexingStreamRecordSerializer + * + * @param The type of value in the {@link StreamRecord} + */ +public class StreamRecordSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; - private final TypeSerializer typeSerializer; - private final boolean isTuple; + protected final TypeSerializer typeSerializer; - public StreamRecordSerializer(TypeInformation typeInfo, ExecutionConfig executionConfig) { - this.typeSerializer = typeInfo.createSerializer(executionConfig); - this.isTuple = typeInfo.isTupleType(); - } - - public TypeSerializer getObjectSerializer() { - return typeSerializer; + public StreamRecordSerializer(TypeSerializer serializer) { + if (serializer instanceof StreamRecordSerializer) { + throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer); + } + this.typeSerializer = Preconditions.checkNotNull(serializer); } @Override @@ -48,34 +57,34 @@ public boolean isImmutableType() { } @Override - public StreamRecordSerializer duplicate() { + @SuppressWarnings("unchecked") + public TypeSerializer duplicate() { return this; } @Override - public StreamRecord createInstance() { + public Object createInstance() { try { - StreamRecord t = new StreamRecord(); - t.isTuple = isTuple; - t.setObject(typeSerializer.createInstance()); - return t; + return new StreamRecord(typeSerializer.createInstance()); } catch (Exception e) { throw new RuntimeException("Cannot instantiate StreamRecord.", e); } } @Override - public StreamRecord copy(StreamRecord from) { - StreamRecord rec = new StreamRecord(); - rec.isTuple = from.isTuple; - rec.setObject(typeSerializer.copy(from.getObject())); - return rec; + @SuppressWarnings("unchecked") + public Object copy(Object from) { + StreamRecord fromRecord = (StreamRecord) from; + return new StreamRecord(typeSerializer.copy(fromRecord.getValue()), fromRecord.getTimestamp()); } @Override - public StreamRecord copy(StreamRecord from, StreamRecord reuse) { - reuse.isTuple = from.isTuple; - reuse.setObject(typeSerializer.copy(from.getObject(), reuse.getObject())); + @SuppressWarnings("unchecked") + public Object copy(Object from, Object reuse) { + StreamRecord fromRecord = (StreamRecord) from; + StreamRecord reuseRecord = (StreamRecord) reuse; + + reuseRecord.replace(typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue()), 0); return reuse; } @@ -85,26 +94,29 @@ public int getLength() { } @Override - public void serialize(StreamRecord value, DataOutputView target) throws IOException { - typeSerializer.serialize(value.getObject(), target); + @SuppressWarnings("unchecked") + public void serialize(Object value, DataOutputView target) throws IOException { + StreamRecord record = (StreamRecord) value; + typeSerializer.serialize(record.getValue(), target); } @Override - public StreamRecord deserialize(DataInputView source) throws IOException { - StreamRecord record = new StreamRecord(); - record.isTuple = this.isTuple; - record.setObject(typeSerializer.deserialize(source)); - return record; + public Object deserialize(DataInputView source) throws IOException { + T element = typeSerializer.deserialize(source); + return new StreamRecord(element, 0); } @Override - public StreamRecord deserialize(StreamRecord reuse, DataInputView source) throws IOException { - reuse.setObject(typeSerializer.deserialize(reuse.getObject(), source)); + @SuppressWarnings("unchecked") + public Object deserialize(Object reuse, DataInputView source) throws IOException { + StreamRecord reuseRecord = (StreamRecord) reuse; + T element = typeSerializer.deserialize(reuseRecord.getValue(), source); + reuseRecord.replace(element, 0); return reuse; } @Override public void copy(DataInputView source, DataOutputView target) throws IOException { - // Needs to be implemented + typeSerializer.copy(source, target); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingSuperstep.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointBarrier.java similarity index 62% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingSuperstep.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointBarrier.java index f749773898df1..d94b5b4b9c49d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingSuperstep.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointBarrier.java @@ -24,14 +24,28 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.event.task.TaskEvent; -public class StreamingSuperstep extends TaskEvent { +/** + * Checkpoint barriers are used to synchronize checkpoints throughout the streaming topology. The + * barriers are emitted by the sources when instructed to do so by the JobManager. When + * operators receive a {@link CheckpointBarrier} on one of its inputs it must block processing + * of further elements on this input until all inputs received the checkpoint barrier + * corresponding to to that checkpoint. Once all inputs received the checkpoint barrier for + * a checkpoint the operator is to perform the checkpoint and then broadcast the barrier to + * downstream operators. + * + *

+ * The checkpoint barrier IDs are advancing. Once an operator receives a {@link CheckpointBarrier} + * for a checkpoint with a higher id it is to discard all barriers that it received from previous + * checkpoints and unblock all other inputs. + */ +public class CheckpointBarrier extends TaskEvent { protected long id; protected long timestamp; - public StreamingSuperstep() {} + public CheckpointBarrier() {} - public StreamingSuperstep(long id, long timestamp) { + public CheckpointBarrier(long id, long timestamp) { this.id = id; this.timestamp = timestamp; } @@ -67,17 +81,17 @@ public int hashCode() { @Override public boolean equals(Object other) { - if (other == null || !(other instanceof StreamingSuperstep)) { + if (other == null || !(other instanceof CheckpointBarrier)) { return false; } else { - StreamingSuperstep that = (StreamingSuperstep) other; + CheckpointBarrier that = (CheckpointBarrier) other; return that.id == this.id && that.timestamp == this.timestamp; } } @Override public String toString() { - return String.format("StreamingSuperstep %d @ %d", id, timestamp); + return String.format("CheckpointBarrier %d @ %d", id, timestamp); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index a9ebf5b917f31..9d6e88e375e8f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -18,17 +18,11 @@ package org.apache.flink.streaming.runtime.tasks; -import java.io.IOException; - +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.io.IndexedMutableReader; -import org.apache.flink.streaming.runtime.io.IndexedReaderIterator; -import org.apache.flink.streaming.runtime.io.InputGateFactory; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; +import org.apache.flink.streaming.runtime.io.StreamInputProcessor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,56 +30,26 @@ public class OneInputStreamTask extends StreamTask inSerializer; - private IndexedMutableReader>> inputs; - protected IndexedReaderIterator> recordIterator; - + private StreamInputProcessor inputProcessor; @Override public void registerInputOutput() { super.registerInputOutput(); - inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader()); + TypeSerializer inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader()); int numberOfInputs = configuration.getNumberOfInputs(); if (numberOfInputs > 0) { - InputGate inputGate = InputGateFactory.createInputGate(getEnvironment().getAllInputGates()); - inputs = new IndexedMutableReader>>(inputGate); + InputGate[] inputGates = getEnvironment().getAllInputGates(); + inputProcessor = new StreamInputProcessor(inputGates, inSerializer, getExecutionConfig().areTimestampsEnabled()); + + inputProcessor.registerTaskEventListener(getCheckpointBarrierListener(), CheckpointBarrier.class); AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); - inputs.setReporter(reporter); - - inputs.registerTaskEventListener(getSuperstepListener(), StreamingSuperstep.class); - - recordIterator = new IndexedReaderIterator>(inputs, inSerializer); - } - } - - /* - * Reads the next record from the reader iterator and stores it in the - * nextRecord variable - */ - protected StreamRecord readNext() throws IOException { - StreamRecord nextRecord = inSerializer.createInstance(); - try { - return recordIterator.next(nextRecord); - } catch (IOException e) { - if (isRunning) { - throw new RuntimeException("Could not read next record.", e); - } else { - // Task already cancelled do nothing - return null; - } - } catch (IllegalStateException e) { - if (isRunning) { - throw new RuntimeException("Could not read next record.", e); - } else { - // Task already cancelled do nothing - return null; - } + inputProcessor.setReporter(reporter); } } @@ -103,10 +67,8 @@ public void invoke() throws Exception { openOperator(); operatorOpen = true; - StreamRecord nextRecord; - while (isRunning && (nextRecord = readNext()) != null) { - headContext.setNextInput(nextRecord.getObject()); - streamOperator.processElement(nextRecord.getObject()); + while (inputProcessor.processInput(streamOperator)) { + // nothing to do, just keep processing } closeOperator(); @@ -123,8 +85,7 @@ public void invoke() throws Exception { if (operatorOpen) { try { closeOperator(); - } - catch (Throwable t) { + } catch (Throwable t) { LOG.warn("Exception while closing operator.", t); } } @@ -134,8 +95,8 @@ public void invoke() throws Exception { finally { this.isRunning = false; // Cleanup - inputs.clearBuffers(); - inputs.cleanup(); + inputProcessor.clearBuffers(); + inputProcessor.cleanup(); outputHandler.flushOutputs(); clearBuffers(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java index 41ee3887e1f81..cf17b3e453d5b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java @@ -24,15 +24,15 @@ import java.util.List; import java.util.Map; -import com.google.common.base.Preconditions; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.api.collector.CollectorWrapper; -import org.apache.flink.streaming.api.collector.StreamOutput; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.CollectorWrapper; +import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamEdge; @@ -41,6 +41,7 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.io.RecordWriterFactory; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; +import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.util.Collector; @@ -53,11 +54,11 @@ public class OutputHandler { private StreamTask vertex; private StreamConfig configuration; private ClassLoader cl; - private Output outerOutput; + private Output> outerOutput; public List> chainedOperators; - private Map> outputMap; + private Map> outputMap; private Map chainedConfigs; private List outEdgesInOrder; @@ -75,7 +76,7 @@ public OutputHandler(StreamTask vertex, Map> ac this.vertex = vertex; this.configuration = new StreamConfig(vertex.getTaskConfiguration()); this.chainedOperators = new ArrayList>(); - this.outputMap = new HashMap>(); + this.outputMap = new HashMap>(); this.cl = vertex.getUserCodeClassLoader(); // We read the chained configs, and the order of record writer @@ -90,7 +91,7 @@ public OutputHandler(StreamTask vertex, Map> ac // We iterate through all the out edges from this job vertex and create // a stream output for (StreamEdge outEdge : outEdgesInOrder) { - StreamOutput streamOutput = createStreamOutput( + RecordWriterOutput streamOutput = createStreamOutput( outEdge, outEdge.getTargetId(), chainedConfigs.get(outEdge.getSourceId()), @@ -108,13 +109,13 @@ public OutputHandler(StreamTask vertex, Map> ac } public void broadcastBarrier(long id, long timestamp) throws IOException, InterruptedException { - StreamingSuperstep barrier = new StreamingSuperstep(id, timestamp); - for (StreamOutput streamOutput : outputMap.values()) { + CheckpointBarrier barrier = new CheckpointBarrier(id, timestamp); + for (RecordWriterOutput streamOutput : outputMap.values()) { streamOutput.broadcastEvent(barrier); } } - public Collection> getOutputs() { + public Collection> getOutputs() { return outputMap.values(); } @@ -134,8 +135,7 @@ public List> getChainedOperators(){ * config */ @SuppressWarnings({"unchecked", "rawtypes"}) - private Output createChainedCollector(StreamConfig chainedTaskConfig, Map> accumulatorMap) { - Preconditions.checkNotNull(accumulatorMap); + private Output> createChainedCollector(StreamConfig chainedTaskConfig, Map> accumulatorMap) { // We create a wrapper that will encapsulate the chained operators and @@ -163,7 +163,7 @@ private Output createChainedCollector(StreamConfig chainedTaskConfig, Map if (chainedTaskConfig.isChainStart()) { // The current task is the first chained task at this vertex so we // return the wrapper - return (Output) wrapper; + return (Output>) wrapper; } else { // The current task is a part of the chain so we get the chainable // operator which will be returned and set it up using the wrapper @@ -177,17 +177,21 @@ private Output createChainedCollector(StreamConfig chainedTaskConfig, Map chainedOperators.add(chainableOperator); if (vertex.getExecutionConfig().isObjectReuseEnabled() || chainableOperator.isInputCopyingDisabled()) { - return new OperatorCollector(chainableOperator); + return new ChainingOutput(chainableOperator); } else { - return new CopyingOperatorCollector( - chainableOperator, - (TypeSerializer) chainedTaskConfig.getTypeSerializerIn1(vertex.getUserCodeClassLoader()).getObjectSerializer()); + StreamRecordSerializer serializerIn1; + if (vertex.getExecutionConfig().areTimestampsEnabled()) { + serializerIn1 = new MultiplexingStreamRecordSerializer(chainedTaskConfig.getTypeSerializerIn1(vertex.getUserCodeClassLoader())); + } else { + serializerIn1 = new StreamRecordSerializer(chainedTaskConfig.getTypeSerializerIn1(vertex.getUserCodeClassLoader())); + } + return new CopyingChainingOutput(chainableOperator, (TypeSerializer>) serializerIn1); } } } - public Output getOutput() { + public Output> getOutput() { return outerOutput; } @@ -201,17 +205,11 @@ public Output getOutput() { * The config of upStream task * @return The created StreamOutput */ - private StreamOutput createStreamOutput(StreamEdge edge, Integer outputVertex, + private RecordWriterOutput createStreamOutput(StreamEdge edge, Integer outputVertex, StreamConfig upStreamConfig, int outputIndex, AccumulatorRegistry.Reporter reporter) { - StreamRecordSerializer outSerializer = upStreamConfig - .getTypeSerializerOut1(vertex.userClassLoader); - SerializationDelegate> outSerializationDelegate = null; + TypeSerializer outSerializer = upStreamConfig.getTypeSerializerOut1(vertex.userClassLoader); - if (outSerializer != null) { - outSerializationDelegate = new SerializationDelegate>(outSerializer); - outSerializationDelegate.setInstance(outSerializer.createInstance()); - } @SuppressWarnings("unchecked") StreamPartitioner outputPartitioner = (StreamPartitioner) edge.getPartitioner(); @@ -223,7 +221,8 @@ private StreamOutput createStreamOutput(StreamEdge edge, Integer outputVe output.setReporter(reporter); - StreamOutput streamOutput = new StreamOutput(output, outSerializationDelegate); + @SuppressWarnings("unchecked") + RecordWriterOutput streamOutput = new RecordWriterOutput((RecordWriter) output, outSerializer, vertex.getExecutionConfig().areTimestampsEnabled()); if (LOG.isTraceEnabled()) { LOG.trace("Partitioner set: {} with {} outputs for {}", outputPartitioner.getClass() @@ -234,27 +233,27 @@ private StreamOutput createStreamOutput(StreamEdge edge, Integer outputVe } public void flushOutputs() throws IOException, InterruptedException { - for (StreamOutput streamOutput : getOutputs()) { + for (RecordWriterOutput streamOutput : getOutputs()) { streamOutput.close(); } } public void clearWriters() { - for (StreamOutput output : outputMap.values()) { + for (RecordWriterOutput output : outputMap.values()) { output.clearBuffers(); } } - private static class OperatorCollector implements Output { - - protected OneInputStreamOperator operator; + private static class ChainingOutput implements Output> { + protected OneInputStreamOperator operator; - public OperatorCollector(OneInputStreamOperator operator) { + public ChainingOutput(OneInputStreamOperator operator) { this.operator = operator; } @Override - public void collect(T record) { + @SuppressWarnings("unchecked") + public void collect(StreamRecord record) { try { operator.getRuntimeContext().setNextInput(record); operator.processElement(record); @@ -267,7 +266,19 @@ public void collect(T record) { } @Override - public final void close() { + public void emitWatermark(Watermark mark) { + try { + operator.processWatermark(mark); + } catch (Exception e) { + if (LOG.isErrorEnabled()) { + LOG.error("Could not forward element to operator: {}", e); + } + throw new RuntimeException(e); + } + } + + @Override + public void close() { try { operator.close(); } catch (Exception e) { @@ -278,17 +289,18 @@ public final void close() { } } - private static class CopyingOperatorCollector extends OperatorCollector { - private final TypeSerializer serializer; + private static class CopyingChainingOutput extends ChainingOutput { + private final TypeSerializer> serializer; - @SuppressWarnings({ "rawtypes", "unchecked" }) - public CopyingOperatorCollector(OneInputStreamOperator operator, TypeSerializer serializer) { + public CopyingChainingOutput(OneInputStreamOperator operator, + TypeSerializer> serializer) { super(operator); this.serializer = serializer; } @Override - public void collect(T record) { + @SuppressWarnings("unchecked") + public void collect(StreamRecord record) { try { operator.getRuntimeContext().setNextInput(record); operator.processElement(serializer.copy(record)); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java index 35b5341aca8f9..1940c11f50c99 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java @@ -18,7 +18,10 @@ package org.apache.flink.streaming.runtime.tasks; +import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +43,8 @@ public class SourceStreamTask extends StreamTask> { @Override public void invoke() throws Exception { + final SourceOutput> output = new SourceOutput>(outputHandler.getOutput(), checkpointLock); + this.isRunning = true; boolean operatorOpen = false; @@ -52,7 +57,7 @@ public void invoke() throws Exception { openOperator(); operatorOpen = true; - streamOperator.run(checkpointLock, outputHandler.getOutput()); + streamOperator.run(checkpointLock, output); closeOperator(); operatorOpen = false; @@ -89,4 +94,34 @@ public void cancel() { super.cancel(); streamOperator.cancel(); } + + private static class SourceOutput implements Output { + private final Output output; + private final Object lockObject; + + public SourceOutput(Output output, Object lockObject) { + this.output = output; + this.lockObject = lockObject; + } + + @Override + public void emitWatermark(Watermark mark) { + synchronized (lockObject) { + output.emitWatermark(mark); + } + } + + @Override + public void collect(T record) { + synchronized (lockObject) { + output.collect(record); + } + + } + + @Override + public void close() { + output.close(); + } + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java index e5d58d37e37e1..1736e525b97d8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java @@ -25,9 +25,10 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; -import org.apache.flink.streaming.api.collector.StreamOutput; +import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.io.BlockingQueueBroker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,7 +77,7 @@ public void invoke() throws Exception { LOG.debug("Iteration source {} invoked", getName()); } - Collection> outputs = outputHandler.getOutputs(); + Collection> outputs = outputHandler.getOutputs(); try { StreamRecord nextRecord; @@ -90,8 +91,8 @@ public void invoke() throws Exception { if (nextRecord == null) { break; } - for (StreamOutput output : outputs) { - ((StreamOutput) output).collect(nextRecord.getObject()); + for (RecordWriterOutput output : outputs) { + ((RecordWriterOutput) output).collect(nextRecord); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java index b6e388990856e..9fbc3a73d3611 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java @@ -20,6 +20,9 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.BlockingQueueBroker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.StringUtils; @@ -43,6 +46,7 @@ public StreamIterationTail() { @Override public void registerInputOutput() { super.registerInputOutput(); + try { iterationId = configuration.getIterationId(); iterationWaitTime = configuration.getIterationWaitTime(); @@ -53,59 +57,33 @@ public void registerInputOutput() { throw new StreamTaskException(String.format( "Cannot register inputs of StreamIterationSink %s", iterationId), e); } + this.streamOperator = new RecordPusher(); } - @Override - public void invoke() throws Exception { - isRunning = true; - - if (LOG.isDebugEnabled()) { - LOG.debug("Iteration sink {} invoked", getName()); - } - - try { - forwardRecords(); + class RecordPusher extends AbstractStreamOperator implements OneInputStreamOperator { + private static final long serialVersionUID = 1L; - if (LOG.isDebugEnabled()) { - LOG.debug("Iteration sink {} invoke finished", getName()); + @Override + public void processElement(StreamRecord record) throws Exception { + try { + if (shouldWait) { + dataChannel.offer(record, iterationWaitTime, TimeUnit.MILLISECONDS); + } else { + dataChannel.put(record); + } + } catch (InterruptedException e) { + if (LOG.isErrorEnabled()) { + LOG.error("Pushing back record at iteration %s failed due to: {}", iterationId, + StringUtils.stringifyException(e)); + } + throw e; } } - catch (Exception e) { - LOG.error("Iteration tail " + getEnvironment().getTaskNameWithSubtasks() + " failed", e); - throw e; - } - finally { - // Cleanup - isRunning = false; - clearBuffers(); - } - } - protected void forwardRecords() throws Exception { - StreamRecord reuse = inSerializer.createInstance(); - while ((reuse = recordIterator.next(reuse)) != null) { - if (!pushToQueue(reuse)) { - break; - } - reuse = inSerializer.createInstance(); + @Override + public void processWatermark(Watermark mark) throws Exception { + // ignore } } - private boolean pushToQueue(StreamRecord record) throws InterruptedException { - try { - if (shouldWait) { - return dataChannel.offer(record, iterationWaitTime, TimeUnit.MILLISECONDS); - } else { - dataChannel.put(record); - return true; - } - } catch (InterruptedException e) { - if (LOG.isErrorEnabled()) { - LOG.error("Pushing back record at iteration %s failed due to: {}", iterationId, - StringUtils.stringifyException(e)); - throw e; - } - return false; - } - } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 4ffc8f58d6b24..286202ffe9e46 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -73,12 +73,12 @@ public abstract class StreamTask> extends Abs protected StreamingRuntimeContext headContext; protected ClassLoader userClassLoader; - - private EventListener superstepListener; + + private EventListener checkpointBarrierListener; public StreamTask() { streamOperator = null; - superstepListener = new SuperstepEventListener(); + checkpointBarrierListener = new CheckpointBarrierListener(); contexts = new ArrayList(); } @@ -171,7 +171,9 @@ private enum StateBackend { protected void openOperator() throws Exception { for (StreamOperator operator : outputHandler.getChainedOperators()) { - operator.open(getTaskConfiguration()); + if (operator != null) { + operator.open(getTaskConfiguration()); + } } } @@ -179,7 +181,10 @@ protected void closeOperator() throws Exception { // We need to close them first to last, since upstream operators in the chain might emit // elements in their close methods. for (int i = outputHandler.getChainedOperators().size()-1; i >= 0; i--) { - outputHandler.getChainedOperators().get(i).close(); + StreamOperator operator = outputHandler.getChainedOperators().get(i); + if (operator != null) { + operator.close(); + } } } @@ -194,8 +199,8 @@ public void cancel() { this.isRunning = false; } - public EventListener getSuperstepListener() { - return this.superstepListener; + public EventListener getCheckpointBarrierListener() { + return this.checkpointBarrierListener; } // ------------------------------------------------------------------------ @@ -305,12 +310,12 @@ public String toString() { // ------------------------------------------------------------------------ - private class SuperstepEventListener implements EventListener { + private class CheckpointBarrierListener implements EventListener { @Override public void onEvent(TaskEvent event) { try { - StreamingSuperstep sStep = (StreamingSuperstep) event; + CheckpointBarrier sStep = (CheckpointBarrier) event; triggerCheckpoint(sStep.getId(), sStep.getTimestamp()); } catch (Exception e) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java index 7eff16ecd5ead..751812432e065 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.streaming.api.state.PartitionedStreamOperatorState; import org.apache.flink.streaming.api.state.StreamOperatorState; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** * Implementation of the {@link RuntimeContext}, created by runtime stream UDF @@ -167,10 +168,10 @@ public Map getOperatorStates() { * Next input of the operator. */ @SuppressWarnings("unchecked") - public void setNextInput(Object nextRecord) { + public void setNextInput(StreamRecord nextRecord) { if (statePartitioner != null) { for (PartitionedStreamOperatorState state : partitionedStates) { - state.setCurrentInput(nextRecord); + state.setCurrentInput(nextRecord.getValue()); } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index 2052877c30041..507b813a4de42 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -21,15 +21,11 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; -import org.apache.flink.streaming.runtime.io.CoReaderIterator; -import org.apache.flink.streaming.runtime.io.CoRecordReader; -import org.apache.flink.streaming.runtime.io.InputGateFactory; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; +import org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,11 +33,41 @@ public class TwoInputStreamTask extends StreamTask inputDeserializer1 = null; - protected StreamRecordSerializer inputDeserializer2 = null; + StreamTwoInputProcessor inputProcessor; - CoRecordReader>, DeserializationDelegate>> coReader; - CoReaderIterator, StreamRecord> coIter; + @Override + public void registerInputOutput() { + super.registerInputOutput(); + + TypeSerializer inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); + TypeSerializer inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); + + int numberOfInputs = configuration.getNumberOfInputs(); + + ArrayList inputList1 = new ArrayList(); + ArrayList inputList2 = new ArrayList(); + + List inEdges = configuration.getInPhysicalEdges(userClassLoader); + + for (int i = 0; i < numberOfInputs; i++) { + int inputType = inEdges.get(i).getTypeNumber(); + InputGate reader = getEnvironment().getInputGate(i); + switch (inputType) { + case 1: + inputList1.add(reader); + break; + case 2: + inputList2.add(reader); + break; + default: + throw new RuntimeException("Invalid input type number: " + inputType); + } + } + + inputProcessor = new StreamTwoInputProcessor(inputList1, inputList2, inputDeserializer1, inputDeserializer2, getExecutionConfig().areTimestampsEnabled()); + + inputProcessor.registerTaskEventListener(getCheckpointBarrierListener(), CheckpointBarrier.class); + } @Override public void invoke() throws Exception { @@ -58,38 +84,8 @@ public void invoke() throws Exception { openOperator(); operatorOpen = true; - int next; - StreamRecord reuse1 = inputDeserializer1.createInstance(); - StreamRecord reuse2 = inputDeserializer2.createInstance(); - - while (isRunning) { - try { - next = coIter.next(reuse1, reuse2); - } catch (IOException e) { - if (isRunning) { - throw new RuntimeException("Could not read next record.", e); - } else { - // Task already cancelled do nothing - next = 0; - } - } catch (IllegalStateException e) { - if (isRunning) { - throw new RuntimeException("Could not read next record.", e); - } else { - // Task already cancelled do nothing - next = 0; - } - } - - if (next == 0) { - break; - } else if (next == 1) { - streamOperator.processElement1(reuse1.getObject()); - reuse1 = inputDeserializer1.createInstance(); - } else { - streamOperator.processElement2(reuse2.getObject()); - reuse2 = inputDeserializer2.createInstance(); - } + while (inputProcessor.processInput(streamOperator)) { + // do nothing, just keep processing } closeOperator(); @@ -123,48 +119,10 @@ public void invoke() throws Exception { } - @Override - public void registerInputOutput() { - super.registerInputOutput(); - - inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); - inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); - - int numberOfInputs = configuration.getNumberOfInputs(); - - ArrayList inputList1 = new ArrayList(); - ArrayList inputList2 = new ArrayList(); - - List inEdges = configuration.getInPhysicalEdges(userClassLoader); - - for (int i = 0; i < numberOfInputs; i++) { - int inputType = inEdges.get(i).getTypeNumber(); - InputGate reader = getEnvironment().getInputGate(i); - switch (inputType) { - case 1: - inputList1.add(reader); - break; - case 2: - inputList2.add(reader); - break; - default: - throw new RuntimeException("Invalid input type number: " + inputType); - } - } - - final InputGate reader1 = InputGateFactory.createInputGate(inputList1); - final InputGate reader2 = InputGateFactory.createInputGate(inputList2); - - coReader = new CoRecordReader>, DeserializationDelegate>>( - reader1, reader2); - coIter = new CoReaderIterator, StreamRecord>(coReader, - inputDeserializer1, inputDeserializer2); - } - @Override public void clearBuffers() throws IOException { super.clearBuffers(); - coReader.clearBuffers(); - coReader.cleanup(); + inputProcessor.clearBuffers(); + inputProcessor.cleanup(); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java new file mode 100644 index 0000000000000..a20436a0f40cb --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -0,0 +1,232 @@ +/* +* 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. +*/ + +// We have it in this package because we could not mock the methods otherwise +package org.apache.flink.runtime.io.network.partition.consumer; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; +import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.plugable.SerializationDelegate; +import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.io.IOException; +import java.util.concurrent.ConcurrentLinkedQueue; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Test {@link InputGate} that allows setting multiple channels. Use + * {@link #sendElement(Object, int)} to offer an element on a specific channel. Use + * {@link #sendEvent(AbstractEvent, int)} to offer an event on the specified channel. Use + * {@link #endInput()} to notify all channels of input end. + */ +public class StreamTestSingleInputGate extends TestSingleInputGate { + + private final int numInputChannels; + + private final TestInputChannel[] inputChannels; + + private final int bufferSize; + + private TypeSerializer serializer; + + private ConcurrentLinkedQueue>[] inputQueues; + + public StreamTestSingleInputGate( + int numInputChannels, + int bufferSize, + TypeSerializer serializer) throws IOException, InterruptedException { + super(numInputChannels, false); + + this.bufferSize = bufferSize; + this.serializer = serializer; + + this.numInputChannels = numInputChannels; + inputChannels = new TestInputChannel[numInputChannels]; + + inputQueues = new ConcurrentLinkedQueue[numInputChannels]; + + setupInputChannels(); + } + + @SuppressWarnings("unchecked") + private void setupInputChannels() throws IOException, InterruptedException { + + for (int i = 0; i < numInputChannels; i++) { + final int channelIndex = i; + final RecordSerializer>> recordSerializer = new SpanningRecordSerializer>>(); + final SerializationDelegate delegate = new SerializationDelegate(new MultiplexingStreamRecordSerializer(serializer)); + + inputQueues[channelIndex] = new ConcurrentLinkedQueue>(); + inputChannels[channelIndex] = new TestInputChannel(inputGate, i); + + + final Answer answer = new Answer() { + @Override + public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable { + InputValue input = inputQueues[channelIndex].poll(); + if (input != null && input.isStreamEnd()) { + when(inputChannels[channelIndex].getInputChannel().isReleased()).thenReturn( + true); + return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE); + } else if (input != null && input.isStreamRecord()) { + Object inputElement = input.getStreamRecord(); + final Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]), + mock(BufferRecycler.class)); + recordSerializer.setNextBuffer(buffer); + delegate.setInstance(inputElement); + recordSerializer.addRecord(delegate); + + // Call getCurrentBuffer to ensure size is set + return recordSerializer.getCurrentBuffer(); + } else if (input != null && input.isEvent()) { + AbstractEvent event = input.getEvent(); + return EventSerializer.toBuffer(event); + } else { + synchronized (inputQueues[channelIndex]) { + inputQueues[channelIndex].wait(); + return answer(invocationOnMock); + } + } + } + }; + + when(inputChannels[channelIndex].getInputChannel().getNextBuffer()).thenAnswer(answer); + + inputGate.setInputChannel(new IntermediateResultPartitionID(), + inputChannels[channelIndex].getInputChannel()); + } + } + + public void sendElement(Object element, int channel) { + synchronized (inputQueues[channel]) { + inputQueues[channel].add(InputValue.element(element)); + inputQueues[channel].notifyAll(); + } + inputGate.onAvailableBuffer(inputChannels[channel].getInputChannel()); + } + + public void sendEvent(AbstractEvent event, int channel) { + synchronized (inputQueues[channel]) { + inputQueues[channel].add(InputValue.event(event)); + inputQueues[channel].notifyAll(); + } + inputGate.onAvailableBuffer(inputChannels[channel].getInputChannel()); + } + + public void endInput() { + for (int i = 0; i < numInputChannels; i++) { + synchronized (inputQueues[i]) { + inputQueues[i].add(InputValue.streamEnd()); + inputQueues[i].notifyAll(); + } + inputGate.onAvailableBuffer(inputChannels[i].getInputChannel()); + } + } + + /** + * Returns true iff all input queues are empty. + */ + public boolean allQueuesEmpty() { +// for (int i = 0; i < numInputChannels; i++) { +// synchronized (inputQueues[i]) { +// inputQueues[i].add(InputValue.event(new DummyEvent())); +// inputQueues[i].notifyAll(); +// inputGate.onAvailableBuffer(inputChannels[i].getInputChannel()); +// } +// } + + for (int i = 0; i < numInputChannels; i++) { + if (inputQueues[i].size() > 0) { + return false; + } + } + return true; + } + + public static class InputValue { + private Object elementOrEvent; + private boolean isStreamEnd; + private boolean isStreamRecord; + private boolean isEvent; + + private InputValue(Object elementOrEvent, boolean isStreamEnd, boolean isEvent, boolean isStreamRecord) { + this.elementOrEvent = elementOrEvent; + this.isStreamEnd = isStreamEnd; + this.isStreamRecord = isStreamRecord; + this.isEvent = isEvent; + } + + public static InputValue element(Object element) { + return new InputValue(element, false, false, true); + } + + public static InputValue streamEnd() { + return new InputValue(null, true, false, false); + } + + public static InputValue event(AbstractEvent event) { + return new InputValue(event, false, true, false); + } + + public Object getStreamRecord() { + return elementOrEvent; + } + + public AbstractEvent getEvent() { + return (AbstractEvent) elementOrEvent; + } + + public boolean isStreamEnd() { + return isStreamEnd; + } + + public boolean isStreamRecord() { + return isStreamRecord; + } + + public boolean isEvent() { + return isEvent; + } + } + + public static class DummyEvent extends TaskEvent { + @Override + public void write(DataOutputView out) throws IOException { + } + + @Override + public void read(DataInputView in) throws IOException { + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java deleted file mode 100644 index 118b23dc10efd..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java +++ /dev/null @@ -1,52 +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.collector; - -import static org.junit.Assert.assertArrayEquals; - -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.api.streamtask.MockRecordWriter; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.MockRecordWriterFactory; -import org.apache.flink.util.Collector; -import org.junit.Test; - -public class StreamCollectorTest { - - @Test - public void testCollect() { - MockRecordWriter recWriter = MockRecordWriterFactory.create(); - SerializationDelegate>> sd = new SerializationDelegate>>( - null); - sd.setInstance(new StreamRecord>().setObject(new Tuple1())); - - Collector> collector = new StreamOutput>(recWriter, sd); - collector.collect(new Tuple1(3)); - collector.collect(new Tuple1(4)); - collector.collect(new Tuple1(5)); - collector.collect(new Tuple1(6)); - - assertArrayEquals(new Integer[] { 3, 4, 5, 6 }, recWriter.emittedRecords.toArray()); - } - - @Test - public void testClose() { - } - -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java index f241955802ab2..e4dadf0f85510 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.functions; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; import java.util.List; @@ -59,8 +60,23 @@ public void collect(T element) { } } + @Override + public void collectWithTimestamp(T element, long timestamp) { + target.add(element); + } + + @Override + public void emitWatermark(Watermark mark) { + // don't do anything + } + @Override public Object getCheckpointLock() { return lock; } + + @Override + public void close() { + + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/CounterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/CounterTest.java deleted file mode 100644 index dbbde2987560a..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/CounterTest.java +++ /dev/null @@ -1,40 +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.operators; - -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.streaming.api.operators.StreamCounter; -import org.apache.flink.streaming.util.MockContext; -import org.junit.Test; - -public class CounterTest { - - @Test - public void counterTest() { - StreamCounter operator = new StreamCounter(); - - List expected = Arrays.asList(1L, 2L, 3L); - List actual = MockContext.createAndExecute(operator, Arrays.asList("one", "two", "three")); - - assertEquals(expected, actual); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/FilterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/FilterTest.java deleted file mode 100644 index ebde006c21169..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/FilterTest.java +++ /dev/null @@ -1,52 +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.operators; - -import static org.junit.Assert.assertEquals; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.streaming.api.operators.StreamFilter; -import org.apache.flink.streaming.util.MockContext; -import org.junit.Test; - -public class FilterTest implements Serializable { - private static final long serialVersionUID = 1L; - - static class MyFilter implements FilterFunction { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(Integer value) throws Exception { - return value % 2 == 0; - } - } - - @Test - public void test() { - StreamFilter operator = new StreamFilter(new MyFilter()); - - List expected = Arrays.asList(2, 4, 6); - List actual = MockContext.createAndExecute(operator, Arrays.asList(1, 2, 3, 4, 5, 6, 7)); - - assertEquals(expected, actual); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/FlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/FlatMapTest.java deleted file mode 100644 index 7f914dddaf590..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/FlatMapTest.java +++ /dev/null @@ -1,55 +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.operators; - -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.streaming.api.operators.StreamFlatMap; -import org.apache.flink.streaming.util.MockContext; -import org.apache.flink.util.Collector; -import org.junit.Test; - -public class FlatMapTest { - - public static final class MyFlatMap implements FlatMapFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void flatMap(Integer value, Collector out) throws Exception { - if (value % 2 == 0) { - out.collect(value); - out.collect(value * value); - } - } - } - - @Test - public void flatMapTest() { - StreamFlatMap operator = new StreamFlatMap(new MyFlatMap()); - - List expected = Arrays.asList(2, 4, 4, 16, 6, 36, 8, 64); - List actual = MockContext.createAndExecute(operator, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8)); - - assertEquals(expected, actual); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/GroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/GroupedFoldTest.java deleted file mode 100644 index 7a45035627780..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/GroupedFoldTest.java +++ /dev/null @@ -1,67 +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.operators; - -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.api.common.functions.FoldFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.streaming.api.operators.StreamGroupedFold; -import org.apache.flink.streaming.util.MockContext; -import org.junit.Test; - -public class GroupedFoldTest { - - private static class MyFolder implements FoldFunction { - - private static final long serialVersionUID = 1L; - - @Override - public String fold(String accumulator, Integer value) throws Exception { - return accumulator + value.toString(); - } - - } - - @Test - public void test() { - TypeInformation outType = TypeExtractor.getForObject("A string"); - - StreamGroupedFold operator1 = new StreamGroupedFold( - new MyFolder(), new KeySelector() { - - private static final long serialVersionUID = 1L; - - @Override - public String getKey(Integer value) throws Exception { - return value.toString(); - } - }, "100", outType); - - List expected = Arrays.asList("1001","10011", "1002", "10022", "1003"); - List actual = MockContext.createAndExecute(operator1, - Arrays.asList(1, 1, 2, 2, 3)); - - assertEquals(expected, actual); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/GroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/GroupedReduceTest.java deleted file mode 100644 index b9e971751efd4..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/GroupedReduceTest.java +++ /dev/null @@ -1,62 +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.operators; - -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.streaming.util.MockContext; -import org.junit.Test; - -public class GroupedReduceTest { - - private static class MyReducer implements ReduceFunction { - - private static final long serialVersionUID = 1L; - - @Override - public Integer reduce(Integer value1, Integer value2) throws Exception { - return value1 + value2; - } - - } - - @Test - public void test() { - StreamGroupedReduce operator1 = new StreamGroupedReduce( - new MyReducer(), new KeySelector() { - - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(Integer value) throws Exception { - return value; - } - }); - - List expected = Arrays.asList(1, 2, 2, 4, 3); - List actual = MockContext.createAndExecute(operator1, - Arrays.asList(1, 1, 2, 2, 3)); - - assertEquals(expected, actual); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/MapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/MapTest.java deleted file mode 100644 index 394b5a4c5039b..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/MapTest.java +++ /dev/null @@ -1,50 +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.operators; - -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.streaming.api.operators.StreamMap; -import org.apache.flink.streaming.util.MockContext; -import org.junit.Test; - -public class MapTest { - - private static class Map implements MapFunction { - private static final long serialVersionUID = 1L; - - @Override - public String map(Integer value) throws Exception { - return "+" + (value + 1); - } - } - - @Test - public void mapTest() { - StreamMap operator = new StreamMap(new Map()); - - List expectedList = Arrays.asList("+2", "+3", "+4"); - List actualList = MockContext.createAndExecute(operator, Arrays.asList(1, 2, 3)); - - assertEquals(expectedList, actualList); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java new file mode 100644 index 0000000000000..3e662badfc450 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.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.operators; + +import java.util.concurrent.ConcurrentLinkedQueue; + +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.joda.time.Instant; +import org.junit.Test; + +/** + * Tests for {@link StreamCounter}. These test that: + * + *
    + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class StreamCounterTest { + + @Test + public void testCount() throws Exception { + StreamCounter operator = new StreamCounter(); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord("eins", initialTime + 1)); + testHarness.processElement(new StreamRecord("zwei", initialTime + 2)); + testHarness.processWatermark(new Watermark(initialTime + 2)); + testHarness.processElement(new StreamRecord("drei", initialTime + 3)); + + expectedOutput.add(new StreamRecord(1L, initialTime + 1)); + expectedOutput.add(new StreamRecord(2L, initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord(3L, initialTime + 3)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java new file mode 100644 index 0000000000000..f672a89663cd4 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java @@ -0,0 +1,135 @@ +/* + * 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.operators; + +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.RichFilterFunction; +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.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link StreamFilter}. These test that: + * + *
    + *
  • RichFunction methods are called correctly
  • + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class StreamFilterTest { + + static class MyFilter implements FilterFunction { + private static final long serialVersionUID = 1L; + + @Override + public boolean filter(Integer value) throws Exception { + return value % 2 == 0; + } + } + + @Test + @SuppressWarnings("unchecked") + public void testFilter() throws Exception { + StreamFilter operator = new StreamFilter(new MyFilter()); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord(1, initialTime + 1)); + testHarness.processElement(new StreamRecord(2, initialTime + 2)); + testHarness.processWatermark(new Watermark(initialTime + 2)); + testHarness.processElement(new StreamRecord(3, initialTime + 3)); + testHarness.processElement(new StreamRecord(4, initialTime + 4)); + testHarness.processElement(new StreamRecord(5, initialTime + 5)); + testHarness.processElement(new StreamRecord(6, initialTime + 6)); + testHarness.processElement(new StreamRecord(7, initialTime + 7)); + + expectedOutput.add(new StreamRecord(2, initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord(4, initialTime + 4)); + expectedOutput.add(new StreamRecord(6, initialTime + 6)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testOpenClose() throws Exception { + StreamFilter operator = new StreamFilter(new TestOpenCloseFilterFunction()); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + + testHarness.open(); + + testHarness.processElement(new StreamRecord("fooHello", initialTime)); + testHarness.processElement(new StreamRecord("bar", initialTime)); + + testHarness.close(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseFilterFunction.closeCalled); + Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseFilterFunction extends RichFilterFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public boolean filter(String value) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return value.startsWith("foo"); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java new file mode 100644 index 0000000000000..ac7caa7762333 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java @@ -0,0 +1,144 @@ +/* + * 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.operators; + +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +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.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link StreamMap}. These test that: + * + *
    + *
  • RichFunction methods are called correctly
  • + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class StreamFlatMapTest { + + public static final class MyFlatMap implements FlatMapFunction { + + private static final long serialVersionUID = 1L; + + @Override + public void flatMap(Integer value, Collector out) throws Exception { + if (value % 2 == 0) { + out.collect(value); + out.collect(value * value); + } + } + } + + @Test + public void testFlatMap() throws Exception { + StreamFlatMap operator = new StreamFlatMap(new MyFlatMap()); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord(1, initialTime + 1)); + testHarness.processElement(new StreamRecord(2, initialTime + 2)); + testHarness.processWatermark(new Watermark(initialTime + 2)); + testHarness.processElement(new StreamRecord(3, initialTime + 3)); + testHarness.processElement(new StreamRecord(4, initialTime + 4)); + testHarness.processElement(new StreamRecord(5, initialTime + 5)); + testHarness.processElement(new StreamRecord(6, initialTime + 6)); + testHarness.processElement(new StreamRecord(7, initialTime + 7)); + testHarness.processElement(new StreamRecord(8, initialTime + 8)); + + expectedOutput.add(new StreamRecord(2, initialTime + 2)); + expectedOutput.add(new StreamRecord(4, initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord(4, initialTime + 4)); + expectedOutput.add(new StreamRecord(16, initialTime + 4)); + expectedOutput.add(new StreamRecord(6, initialTime + 6)); + expectedOutput.add(new StreamRecord(36, initialTime + 6)); + expectedOutput.add(new StreamRecord(8, initialTime + 8)); + expectedOutput.add(new StreamRecord(64, initialTime + 8)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testOpenClose() throws Exception { + StreamFlatMap operator = new StreamFlatMap(new TestOpenCloseFlatMapFunction()); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + + testHarness.open(); + + testHarness.processElement(new StreamRecord("Hello", initialTime)); + + testHarness.close(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseFlatMapFunction.closeCalled); + Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseFlatMapFunction extends RichFlatMapFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public void flatMap(String value, Collector out) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + out.collect(value); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java new file mode 100644 index 0000000000000..8499aa2aa9d0d --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java @@ -0,0 +1,159 @@ +/* + * 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.operators; + +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.flink.api.common.functions.FoldFunction; +import org.apache.flink.api.common.functions.RichFoldFunction; +import org.apache.flink.api.common.functions.RichReduceFunction; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.TypeExtractor; +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.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link StreamGroupedFold}. These test that: + * + *
    + *
  • RichFunction methods are called correctly
  • + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class StreamGroupedFoldTest { + + private static class MyFolder implements FoldFunction { + + private static final long serialVersionUID = 1L; + + @Override + public String fold(String accumulator, Integer value) throws Exception { + return accumulator + value.toString(); + } + + } + + @Test + @SuppressWarnings("unchecked") + public void testGroupedFold() throws Exception { + TypeInformation outType = TypeExtractor.getForObject("A string"); + + StreamGroupedFold operator = new StreamGroupedFold( + new MyFolder(), new KeySelector() { + + private static final long serialVersionUID = 1L; + + @Override + public String getKey(Integer value) throws Exception { + return value.toString(); + } + }, "100", outType); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord(1, initialTime + 1)); + testHarness.processElement(new StreamRecord(1, initialTime + 2)); + testHarness.processWatermark(new Watermark(initialTime + 2)); + testHarness.processElement(new StreamRecord(2, initialTime + 3)); + testHarness.processElement(new StreamRecord(2, initialTime + 4)); + testHarness.processElement(new StreamRecord(3, initialTime + 5)); + + expectedOutput.add(new StreamRecord("1001", initialTime + 1)); + expectedOutput.add(new StreamRecord("10011", initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord("1002", initialTime + 3)); + expectedOutput.add(new StreamRecord("10022", initialTime + 4)); + expectedOutput.add(new StreamRecord("1003", initialTime + 5)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testOpenClose() throws Exception { + StreamGroupedFold operator = new StreamGroupedFold(new TestOpenCloseFoldFunction(), new KeySelector() { + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + }, "init", BasicTypeInfo.STRING_TYPE_INFO); + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + + testHarness.open(); + + testHarness.processElement(new StreamRecord(1, initialTime)); + testHarness.processElement(new StreamRecord(2, initialTime)); + + testHarness.close(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseFoldFunction.closeCalled); + Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseFoldFunction extends RichFoldFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public String fold(String acc, Integer in) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return acc + in; + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java new file mode 100644 index 0000000000000..dca1cbb1a3854 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java @@ -0,0 +1,153 @@ +/* + * 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.operators; + + +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.functions.RichReduceFunction; +import org.apache.flink.api.java.functions.KeySelector; +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.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link StreamGroupedReduce}. These test that: + * + *
    + *
  • RichFunction methods are called correctly
  • + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ + +public class StreamGroupedReduceTest { + + private static class MyReducer implements ReduceFunction { + + private static final long serialVersionUID = 1L; + + @Override + public Integer reduce(Integer value1, Integer value2) throws Exception { + return value1 + value2; + } + + } + + @Test + @SuppressWarnings("unchecked") + public void testGroupedReduce() throws Exception { + StreamGroupedReduce operator = new StreamGroupedReduce(new MyReducer(), new KeySelector() { + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + }); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord(1, initialTime + 1)); + testHarness.processElement(new StreamRecord(1, initialTime + 2)); + testHarness.processWatermark(new Watermark(initialTime + 2)); + testHarness.processElement(new StreamRecord(2, initialTime + 3)); + testHarness.processElement(new StreamRecord(2, initialTime + 4)); + testHarness.processElement(new StreamRecord(3, initialTime + 5)); + + expectedOutput.add(new StreamRecord(1, initialTime + 1)); + expectedOutput.add(new StreamRecord(2, initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord(2, initialTime + 3)); + expectedOutput.add(new StreamRecord(4, initialTime + 4)); + expectedOutput.add(new StreamRecord(3, initialTime + 5)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testOpenClose() throws Exception { + StreamGroupedReduce operator = new StreamGroupedReduce(new TestOpenCloseReduceFunction(), new KeySelector() { + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + }); + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + + testHarness.open(); + + testHarness.processElement(new StreamRecord(1, initialTime)); + testHarness.processElement(new StreamRecord(2, initialTime)); + + testHarness.close(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseReduceFunction.closeCalled); + Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseReduceFunction extends RichReduceFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public Integer reduce(Integer in1, Integer in2) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return in1 + in2; + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java new file mode 100644 index 0000000000000..d5f2f62bdcd72 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java @@ -0,0 +1,129 @@ +/* + * 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.operators; + +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +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.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link StreamMap}. These test that: + * + *
    + *
  • RichFunction methods are called correctly
  • + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class StreamMapTest { + + private static class Map implements MapFunction { + private static final long serialVersionUID = 1L; + + @Override + public String map(Integer value) throws Exception { + return "+" + (value + 1); + } + } + + @Test + public void testMap() throws Exception { + StreamMap operator = new StreamMap(new Map()); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord(1, initialTime + 1)); + testHarness.processElement(new StreamRecord(2, initialTime + 2)); + testHarness.processWatermark(new Watermark(initialTime + 2)); + testHarness.processElement(new StreamRecord(3, initialTime + 3)); + + expectedOutput.add(new StreamRecord("+2", initialTime + 1)); + expectedOutput.add(new StreamRecord("+3", initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord("+4", initialTime + 3)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testOpenClose() throws Exception { + StreamMap operator = new StreamMap(new TestOpenCloseMapFunction()); + + OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + + testHarness.open(); + + testHarness.processElement(new StreamRecord("Hello", initialTime)); + + testHarness.close(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseMapFunction.closeCalled); + Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseMapFunction extends RichMapFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public String map(String value) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return value; + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/ProjectTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java similarity index 59% rename from flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/ProjectTest.java rename to flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java index d9cc607fb4d26..ede7db50e2d48 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/ProjectTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java @@ -21,9 +21,8 @@ import static org.junit.Assert.fail; import java.io.Serializable; -import java.util.ArrayList; import java.util.HashSet; -import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.MapFunction; @@ -38,15 +37,27 @@ import org.apache.flink.streaming.api.datastream.StreamProjection; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.util.MockContext; +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.streaming.util.TestStreamEnvironment; +import org.joda.time.Instant; import org.junit.Test; -public class ProjectTest implements Serializable { +/** + * Tests for {@link StreamProject}. These test that: + * + *
    + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class StreamProjectTest implements Serializable { private static final long serialVersionUID = 1L; @Test - public void operatorTest() { + public void testProject() throws Exception { TypeInformation> inType = TypeExtractor .getForObject(new Tuple5(2, "a", 3, "b", 4)); @@ -61,20 +72,26 @@ public void operatorTest() { new StreamProject, Tuple3>( fields, serializer); - List> input = new ArrayList>(); - input.add(new Tuple5(2, "a", 3, "b", 4)); - input.add(new Tuple5(2, "s", 3, "c", 2)); - input.add(new Tuple5(2, "a", 3, "c", 2)); - input.add(new Tuple5(2, "a", 3, "a", 7)); + OneInputStreamOperatorTestHarness, Tuple3> testHarness = new OneInputStreamOperatorTestHarness, Tuple3>(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord>(new Tuple5(2, "a", 3, "b", 4), initialTime + 1)); + testHarness.processElement(new StreamRecord>(new Tuple5(2, "s", 3, "c", 2), initialTime + 2)); + testHarness.processElement(new StreamRecord>(new Tuple5(2, "a", 3, "c", 2), initialTime + 3)); + testHarness.processWatermark(new Watermark(initialTime + 2)); + testHarness.processElement(new StreamRecord>(new Tuple5(2, "a", 3, "a", 7), initialTime + 4)); - List> expected = new ArrayList>(); - expected.add(new Tuple3(4, 4, "b")); - expected.add(new Tuple3(2, 2, "c")); - expected.add(new Tuple3(2, 2, "c")); - expected.add(new Tuple3(7, 7, "a")); + expectedOutput.add(new StreamRecord>(new Tuple3(4, 4, "b"), initialTime + 1)); + expectedOutput.add(new StreamRecord>(new Tuple3(2, 2, "c"), initialTime + 2)); + expectedOutput.add(new StreamRecord>(new Tuple3(2, 2, "c"), initialTime + 3)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord>(new Tuple3(7, 7, "a"), initialTime + 4)); - assertEquals(expected, MockContext.createAndExecute(operator, input)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoFlatMapTest.java deleted file mode 100644 index 7f23e2332dc3d..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoFlatMapTest.java +++ /dev/null @@ -1,83 +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.operators.co; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; -import org.apache.flink.streaming.util.MockCoContext; -import org.apache.flink.util.Collector; -import org.junit.Test; - -public class CoFlatMapTest implements Serializable { - private static final long serialVersionUID = 1L; - - private final static class MyCoFlatMap implements CoFlatMapFunction { - private static final long serialVersionUID = 1L; - - @Override - public void flatMap1(String value, Collector coll) { - for (int i = 0; i < value.length(); i++) { - coll.collect(value.substring(i, i + 1)); - } - } - - @Override - public void flatMap2(Integer value, Collector coll) { - coll.collect(value.toString()); - } - } - - @Test - public void coFlatMapTest() { - CoStreamFlatMap invokable = new CoStreamFlatMap( - new MyCoFlatMap()); - - List expectedList = Arrays.asList("a", "b", "c", "1", "d", "e", "f", "2", "g", "h", - "e", "3", "4", "5"); - List actualList = MockCoContext.createAndExecute(invokable, - Arrays.asList("abc", "def", "ghe"), Arrays.asList(1, 2, 3, 4, 5)); - - assertEquals(expectedList, actualList); - } - - @SuppressWarnings("unchecked") - @Test - public void multipleInputTest() { - LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1); - - DataStream ds1 = env.fromElements(1, 3, 5); - DataStream ds2 = env.fromElements(2, 4).union(ds1); - - try { - ds1.forward().union(ds2); - fail(); - } catch (RuntimeException e) { - // expected - } - - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java index d01d0d3362df9..39e85e9892cb6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java @@ -1,125 +1,125 @@ -/* - * 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.operators.co; - -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; -import java.util.List; - -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.functions.co.CoReduceFunction; -import org.apache.flink.streaming.api.operators.co.CoStreamGroupedReduce; -import org.apache.flink.streaming.util.MockCoContext; -import org.junit.Test; - -public class CoGroupedReduceTest { - - private final static class MyCoReduceFunction implements - CoReduceFunction, Tuple2, String> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple3 reduce1(Tuple3 value1, - Tuple3 value2) { - return new Tuple3(value1.f0, value1.f1 + value2.f1, value1.f2); - } - - @Override - public Tuple2 reduce2(Tuple2 value1, - Tuple2 value2) { - return new Tuple2(value1.f0, value1.f1 + value2.f1); - } - - @Override - public String map1(Tuple3 value) { - return value.f1; - } - - @Override - public String map2(Tuple2 value) { - return value.f1.toString(); - } - } - - @SuppressWarnings("unchecked") - @Test - public void coGroupedReduceTest() { - Tuple3 word1 = new Tuple3("a", "word1", "b"); - Tuple3 word2 = new Tuple3("b", "word2", "a"); - Tuple3 word3 = new Tuple3("a", "word3", "a"); - Tuple2 int1 = new Tuple2(2, 1); - Tuple2 int2 = new Tuple2(1, 2); - Tuple2 int3 = new Tuple2(0, 3); - Tuple2 int4 = new Tuple2(2, 4); - Tuple2 int5 = new Tuple2(1, 5); - - KeySelector, ?> keySelector0 = new KeySelector, String>() { - - private static final long serialVersionUID = 1L; - - @Override - public String getKey(Tuple3 value) throws Exception { - return value.f0; - } - }; - - KeySelector, ?> keySelector1 = new KeySelector, Integer>() { - - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(Tuple2 value) throws Exception { - return value.f0; - } - }; - - KeySelector, ?> keySelector2 = new KeySelector, String>() { - - private static final long serialVersionUID = 1L; - - @Override - public String getKey(Tuple3 value) throws Exception { - return value.f2; - } - }; - - CoStreamGroupedReduce, Tuple2, String> invokable = new CoStreamGroupedReduce, Tuple2, String>( - new MyCoReduceFunction(), keySelector0, keySelector1); - - List expected = Arrays.asList("word1", "1", "word2", "2", "word1word3", "3", "5", - "7"); - - List actualList = MockCoContext.createAndExecute(invokable, - Arrays.asList(word1, word2, word3), Arrays.asList(int1, int2, int3, int4, int5)); - - assertEquals(expected, actualList); - - invokable = new CoStreamGroupedReduce, Tuple2, String>( - new MyCoReduceFunction(), keySelector2, keySelector1); - - expected = Arrays.asList("word1", "1", "word2", "2", "word2word3", "3", "5", "7"); - - actualList = MockCoContext.createAndExecute(invokable, Arrays.asList(word1, word2, word3), - Arrays.asList(int1, int2, int3, int4, int5)); - - assertEquals(expected, actualList); - } -} +///* +// * 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.operators.co; +// +//import static org.junit.Assert.assertEquals; +// +//import java.util.Arrays; +//import java.util.List; +// +//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.functions.co.CoReduceFunction; +//import org.apache.flink.streaming.api.operators.co.CoStreamGroupedReduce; +//import org.apache.flink.streaming.util.MockCoContext; +//import org.junit.Test; +// +//public class CoGroupedReduceTest { +// +// private final static class MyCoReduceFunction implements +// CoReduceFunction, Tuple2, String> { +// private static final long serialVersionUID = 1L; +// +// @Override +// public Tuple3 reduce1(Tuple3 value1, +// Tuple3 value2) { +// return new Tuple3(value1.f0, value1.f1 + value2.f1, value1.f2); +// } +// +// @Override +// public Tuple2 reduce2(Tuple2 value1, +// Tuple2 value2) { +// return new Tuple2(value1.f0, value1.f1 + value2.f1); +// } +// +// @Override +// public String map1(Tuple3 value) { +// return value.f1; +// } +// +// @Override +// public String map2(Tuple2 value) { +// return value.f1.toString(); +// } +// } +// +// @SuppressWarnings("unchecked") +// @Test +// public void coGroupedReduceTest() { +// Tuple3 word1 = new Tuple3("a", "word1", "b"); +// Tuple3 word2 = new Tuple3("b", "word2", "a"); +// Tuple3 word3 = new Tuple3("a", "word3", "a"); +// Tuple2 int1 = new Tuple2(2, 1); +// Tuple2 int2 = new Tuple2(1, 2); +// Tuple2 int3 = new Tuple2(0, 3); +// Tuple2 int4 = new Tuple2(2, 4); +// Tuple2 int5 = new Tuple2(1, 5); +// +// KeySelector, ?> keySelector0 = new KeySelector, String>() { +// +// private static final long serialVersionUID = 1L; +// +// @Override +// public String getKey(Tuple3 value) throws Exception { +// return value.f0; +// } +// }; +// +// KeySelector, ?> keySelector1 = new KeySelector, Integer>() { +// +// private static final long serialVersionUID = 1L; +// +// @Override +// public Integer getKey(Tuple2 value) throws Exception { +// return value.f0; +// } +// }; +// +// KeySelector, ?> keySelector2 = new KeySelector, String>() { +// +// private static final long serialVersionUID = 1L; +// +// @Override +// public String getKey(Tuple3 value) throws Exception { +// return value.f2; +// } +// }; +// +// CoStreamGroupedReduce, Tuple2, String> invokable = new CoStreamGroupedReduce, Tuple2, String>( +// new MyCoReduceFunction(), keySelector0, keySelector1); +// +// List expected = Arrays.asList("word1", "1", "word2", "2", "word1word3", "3", "5", +// "7"); +// +// List actualList = MockCoContext.createAndExecute(invokable, +// Arrays.asList(word1, word2, word3), Arrays.asList(int1, int2, int3, int4, int5)); +// +// assertEquals(expected, actualList); +// +// invokable = new CoStreamGroupedReduce, Tuple2, String>( +// new MyCoReduceFunction(), keySelector2, keySelector1); +// +// expected = Arrays.asList("word1", "1", "word2", "2", "word2word3", "3", "5", "7"); +// +// actualList = MockCoContext.createAndExecute(invokable, Arrays.asList(word1, word2, word3), +// Arrays.asList(int1, int2, int3, int4, int5)); +// +// assertEquals(expected, actualList); +// } +//} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoMapTest.java deleted file mode 100644 index 2a2560d770803..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoMapTest.java +++ /dev/null @@ -1,57 +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.operators.co; - -import static org.junit.Assert.assertEquals; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - -import org.apache.flink.streaming.api.functions.co.CoMapFunction; -import org.apache.flink.streaming.api.operators.co.CoStreamMap; -import org.apache.flink.streaming.util.MockCoContext; -import org.junit.Test; - -public class CoMapTest implements Serializable { - private static final long serialVersionUID = 1L; - - private final static class MyCoMap implements CoMapFunction { - private static final long serialVersionUID = 1L; - - @Override - public String map1(Double value) { - return value.toString(); - } - - @Override - public String map2(Integer value) { - return value.toString(); - } - } - - @Test - public void coMapTest() { - CoStreamMap invokable = new CoStreamMap(new MyCoMap()); - - List expectedList = Arrays.asList("1.1", "1", "1.2", "2", "1.3", "3", "1.4", "1.5"); - List actualList = MockCoContext.createAndExecute(invokable, Arrays.asList(1.1, 1.2, 1.3, 1.4, 1.5), Arrays.asList(1, 2, 3)); - - assertEquals(expectedList, actualList); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java new file mode 100644 index 0000000000000..2c9ba5c3da77c --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java @@ -0,0 +1,190 @@ +/* + * 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.operators.co; + +import static org.junit.Assert.fail; + +import java.io.Serializable; +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; +import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link CoStreamFlatMap}. These test that: + * + *
    + *
  • RichFunction methods are called correctly
  • + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class CoStreamFlatMapTest implements Serializable { + private static final long serialVersionUID = 1L; + + private final static class MyCoFlatMap implements CoFlatMapFunction { + private static final long serialVersionUID = 1L; + + @Override + public void flatMap1(String value, Collector coll) { + for (int i = 0; i < value.length(); i++) { + coll.collect(value.substring(i, i + 1)); + } + } + + @Override + public void flatMap2(Integer value, Collector coll) { + coll.collect(value.toString()); + } + } + + @Test + public void testCoFlatMap() throws Exception { + CoStreamFlatMap operator = new CoStreamFlatMap(new MyCoFlatMap()); + + TwoInputStreamOperatorTestHarness testHarness = new TwoInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement1(new StreamRecord("abc", initialTime + 1)); + testHarness.processElement1(new StreamRecord("def", initialTime + 2)); + testHarness.processWatermark1(new Watermark(initialTime + 2)); + testHarness.processElement1(new StreamRecord("ghi", initialTime + 3)); + + testHarness.processElement2(new StreamRecord(1, initialTime + 1)); + testHarness.processElement2(new StreamRecord(2, initialTime + 2)); + testHarness.processWatermark2(new Watermark(initialTime + 3)); + testHarness.processElement2(new StreamRecord(3, initialTime + 3)); + testHarness.processElement2(new StreamRecord(4, initialTime + 4)); + testHarness.processElement2(new StreamRecord(5, initialTime + 5)); + + expectedOutput.add(new StreamRecord("a", initialTime + 1)); + expectedOutput.add(new StreamRecord("b", initialTime + 1)); + expectedOutput.add(new StreamRecord("c", initialTime + 1)); + expectedOutput.add(new StreamRecord("d", initialTime + 2)); + expectedOutput.add(new StreamRecord("e", initialTime + 2)); + expectedOutput.add(new StreamRecord("f", initialTime + 2)); + expectedOutput.add(new StreamRecord("g", initialTime + 3)); + expectedOutput.add(new StreamRecord("h", initialTime + 3)); + expectedOutput.add(new StreamRecord("i", initialTime + 3)); + + expectedOutput.add(new StreamRecord("1", initialTime + 1)); + expectedOutput.add(new StreamRecord("2", initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord("3", initialTime + 3)); + expectedOutput.add(new StreamRecord("4", initialTime + 4)); + expectedOutput.add(new StreamRecord("5", initialTime + 5)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testOpenClose() throws Exception { + CoStreamFlatMap operator = new CoStreamFlatMap(new TestOpenCloseCoFlatMapFunction()); + + TwoInputStreamOperatorTestHarness testHarness = new TwoInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + + testHarness.open(); + + testHarness.processElement1(new StreamRecord("Hello", initialTime)); + testHarness.processElement2(new StreamRecord(42, initialTime)); + + testHarness.close(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseCoFlatMapFunction.closeCalled); + Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseCoFlatMapFunction extends RichCoFlatMapFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public void flatMap1(String value, Collector out) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + out.collect(value); + } + + @Override + public void flatMap2(Integer value, Collector out) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + out.collect(value.toString()); + } + + } + + @SuppressWarnings("unchecked") + @Test + public void multipleInputTest() { + LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1); + + DataStream ds1 = env.fromElements(1, 3, 5); + DataStream ds2 = env.fromElements(2, 4).union(ds1); + + try { + ds1.forward().union(ds2); + fail(); + } catch (RuntimeException e) { + // expected + } + + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java new file mode 100644 index 0000000000000..dcf49724bc787 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java @@ -0,0 +1,172 @@ +/* + * 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.operators.co; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; +import org.apache.flink.streaming.api.functions.co.CoMapFunction; +import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; +import org.apache.flink.streaming.api.functions.co.RichCoMapFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Serializable; +import java.util.concurrent.ConcurrentLinkedQueue; + +import static org.junit.Assert.fail; + +/** + * Tests for {@link org.apache.flink.streaming.api.operators.co.CoStreamMap}. These test that: + * + *
    + *
  • RichFunction methods are called correctly
  • + *
  • Timestamps of processed elements match the input timestamp
  • + *
  • Watermarks are correctly forwarded
  • + *
+ */ +public class CoStreamMapTest implements Serializable { + private static final long serialVersionUID = 1L; + + private final static class MyCoMap implements CoMapFunction { + private static final long serialVersionUID = 1L; + + @Override + public String map1(Double value) { + return value.toString(); + } + + @Override + public String map2(Integer value) { + return value.toString(); + } + } + + + @Test + public void testCoMap() throws Exception { + CoStreamMap operator = new CoStreamMap(new MyCoMap()); + + TwoInputStreamOperatorTestHarness testHarness = new TwoInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.open(); + + testHarness.processElement1(new StreamRecord(1.1d, initialTime + 1)); + testHarness.processElement1(new StreamRecord(1.2d, initialTime + 2)); + testHarness.processElement1(new StreamRecord(1.3d, initialTime + 3)); + testHarness.processWatermark1(new Watermark(initialTime + 3)); + testHarness.processElement1(new StreamRecord(1.4d, initialTime + 4)); + testHarness.processElement1(new StreamRecord(1.5d, initialTime + 5)); + + testHarness.processElement2(new StreamRecord(1, initialTime + 1)); + testHarness.processElement2(new StreamRecord(2, initialTime + 2)); + testHarness.processWatermark2(new Watermark(initialTime + 2)); + testHarness.processElement2(new StreamRecord(3, initialTime + 3)); + testHarness.processElement2(new StreamRecord(4, initialTime + 4)); + testHarness.processElement2(new StreamRecord(5, initialTime + 5)); + + expectedOutput.add(new StreamRecord("1.1", initialTime + 1)); + expectedOutput.add(new StreamRecord("1.2", initialTime + 2)); + expectedOutput.add(new StreamRecord("1.3", initialTime + 3)); + expectedOutput.add(new StreamRecord("1.4", initialTime + 4)); + expectedOutput.add(new StreamRecord("1.5", initialTime + 5)); + + expectedOutput.add(new StreamRecord("1", initialTime + 1)); + expectedOutput.add(new StreamRecord("2", initialTime + 2)); + expectedOutput.add(new Watermark(initialTime + 2)); + expectedOutput.add(new StreamRecord("3", initialTime + 3)); + expectedOutput.add(new StreamRecord("4", initialTime + 4)); + expectedOutput.add(new StreamRecord("5", initialTime + 5)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testOpenClose() throws Exception { + CoStreamMap operator = new CoStreamMap(new TestOpenCloseCoMapFunction()); + + TwoInputStreamOperatorTestHarness testHarness = new TwoInputStreamOperatorTestHarness(operator); + + long initialTime = 0L; + + testHarness.open(); + + testHarness.processElement1(new StreamRecord(74d, initialTime)); + testHarness.processElement2(new StreamRecord(42, initialTime)); + + testHarness.close(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseCoMapFunction.closeCalled); + Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseCoMapFunction extends RichCoMapFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public String map1(Double value) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return value.toString(); + } + + @Override + public String map2(Integer value) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return value.toString(); + } + + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java index c0f49c7b57016..130842e8a49b4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java @@ -1,182 +1,182 @@ -/* - * 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.operators.co; - -import static org.junit.Assert.assertEquals; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.functions.co.CoWindowFunction; -import org.apache.flink.streaming.api.operators.co.CoStreamWindow; -import org.apache.flink.streaming.api.windowing.helper.Timestamp; -import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; -import org.apache.flink.streaming.util.MockCoContext; -import org.apache.flink.util.Collector; -import org.junit.Test; - -public class CoWindowTest { - - public static final class MyCoGroup1 implements CoWindowFunction { - - private static final long serialVersionUID = 1L; - - @SuppressWarnings("unused") - @Override - public void coWindow(List first, List second, Collector out) - throws Exception { - Integer count1 = 0; - for (Integer i : first) { - count1++; - } - Integer count2 = 0; - for (Integer i : second) { - count2++; - } - out.collect(count1); - out.collect(count2); - - } - - } - - public static final class MyCoGroup2 implements - CoWindowFunction, Tuple2, Integer> { - - private static final long serialVersionUID = 1L; - - @Override - public void coWindow(List> first, - List> second, Collector out) throws Exception { - - Set firstElements = new HashSet(); - for (Tuple2 value : first) { - firstElements.add(value.f1); - } - for (Tuple2 value : second) { - if (firstElements.contains(value.f1)) { - out.collect(value.f1); - } - } - - } - - } - - private static final class MyTS1 implements Timestamp { - - private static final long serialVersionUID = 1L; - - @Override - public long getTimestamp(Integer value) { - return value; - } - - } - - private static final class MyTS2 implements Timestamp> { - - private static final long serialVersionUID = 1L; - - @Override - public long getTimestamp(Tuple2 value) { - return value.f0; - } - - } - - @Test - public void coWindowGroupReduceTest2() throws Exception { - - CoStreamWindow invokable1 = new CoStreamWindow( - new MyCoGroup1(), 2, 1, new TimestampWrapper(new MyTS1(), 1), - new TimestampWrapper(new MyTS1(), 1)); - - // Windowsize 2, slide 1 - // 1,2|2,3|3,4|4,5 - - List input11 = new ArrayList(); - input11.add(1); - input11.add(1); - input11.add(2); - input11.add(3); - input11.add(3); - - List input12 = new ArrayList(); - input12.add(1); - input12.add(2); - input12.add(3); - input12.add(3); - input12.add(5); - - // Windows: (1,1,2)(1,1,2)|(2,3,3)(2,3,3)|(3,3)(3,3)|(5)(5) - // expected output: 3,2|3,3|2,2|0,1 - - List expected1 = new ArrayList(); - expected1.add(3); - expected1.add(2); - expected1.add(3); - expected1.add(3); - expected1.add(2); - expected1.add(2); - expected1.add(0); - expected1.add(1); - - List actual1 = MockCoContext.createAndExecute(invokable1, input11, input12); - assertEquals(expected1, actual1); - - CoStreamWindow, Tuple2, Integer> invokable2 = new CoStreamWindow, Tuple2, Integer>( - new MyCoGroup2(), 2, 3, new TimestampWrapper>(new MyTS2(), - 1), new TimestampWrapper>(new MyTS2(), 1)); - - // WindowSize 2, slide 3 - // 1,2|4,5|7,8| - - List> input21 = new ArrayList>(); - input21.add(new Tuple2(1, 1)); - input21.add(new Tuple2(1, 2)); - input21.add(new Tuple2(2, 3)); - input21.add(new Tuple2(3, 4)); - input21.add(new Tuple2(3, 5)); - input21.add(new Tuple2(4, 6)); - input21.add(new Tuple2(4, 7)); - input21.add(new Tuple2(5, 8)); - - List> input22 = new ArrayList>(); - input22.add(new Tuple2(1, 1)); - input22.add(new Tuple2(2, 0)); - input22.add(new Tuple2(2, 2)); - input22.add(new Tuple2(3, 9)); - input22.add(new Tuple2(3, 4)); - input22.add(new Tuple2(4, 10)); - input22.add(new Tuple2(5, 8)); - input22.add(new Tuple2(5, 7)); - - List expected2 = new ArrayList(); - expected2.add(1); - expected2.add(2); - expected2.add(8); - expected2.add(7); - - List actual2 = MockCoContext.createAndExecute(invokable2, input21, input22); - assertEquals(expected2, actual2); - } -} +///* +// * 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.operators.co; +// +//import static org.junit.Assert.assertEquals; +// +//import java.util.ArrayList; +//import java.util.HashSet; +//import java.util.List; +//import java.util.Set; +// +//import org.apache.flink.api.java.tuple.Tuple2; +//import org.apache.flink.streaming.api.functions.co.CoWindowFunction; +//import org.apache.flink.streaming.api.operators.co.CoStreamWindow; +//import org.apache.flink.streaming.api.windowing.helper.Timestamp; +//import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; +//import org.apache.flink.streaming.util.MockCoContext; +//import org.apache.flink.util.Collector; +//import org.junit.Test; +// +//public class CoWindowTest { +// +// public static final class MyCoGroup1 implements CoWindowFunction { +// +// private static final long serialVersionUID = 1L; +// +// @SuppressWarnings("unused") +// @Override +// public void coWindow(List first, List second, Collector out) +// throws Exception { +// Integer count1 = 0; +// for (Integer i : first) { +// count1++; +// } +// Integer count2 = 0; +// for (Integer i : second) { +// count2++; +// } +// out.collect(count1); +// out.collect(count2); +// +// } +// +// } +// +// public static final class MyCoGroup2 implements +// CoWindowFunction, Tuple2, Integer> { +// +// private static final long serialVersionUID = 1L; +// +// @Override +// public void coWindow(List> first, +// List> second, Collector out) throws Exception { +// +// Set firstElements = new HashSet(); +// for (Tuple2 value : first) { +// firstElements.add(value.f1); +// } +// for (Tuple2 value : second) { +// if (firstElements.contains(value.f1)) { +// out.collect(value.f1); +// } +// } +// +// } +// +// } +// +// private static final class MyTS1 implements Timestamp { +// +// private static final long serialVersionUID = 1L; +// +// @Override +// public long getTimestamp(Integer value) { +// return value; +// } +// +// } +// +// private static final class MyTS2 implements Timestamp> { +// +// private static final long serialVersionUID = 1L; +// +// @Override +// public long getTimestamp(Tuple2 value) { +// return value.f0; +// } +// +// } +// +// @Test +// public void coWindowGroupReduceTest2() throws Exception { +// +// CoStreamWindow invokable1 = new CoStreamWindow( +// new MyCoGroup1(), 2, 1, new TimestampWrapper(new MyTS1(), 1), +// new TimestampWrapper(new MyTS1(), 1)); +// +// // Windowsize 2, slide 1 +// // 1,2|2,3|3,4|4,5 +// +// List input11 = new ArrayList(); +// input11.add(1); +// input11.add(1); +// input11.add(2); +// input11.add(3); +// input11.add(3); +// +// List input12 = new ArrayList(); +// input12.add(1); +// input12.add(2); +// input12.add(3); +// input12.add(3); +// input12.add(5); +// +// // Windows: (1,1,2)(1,1,2)|(2,3,3)(2,3,3)|(3,3)(3,3)|(5)(5) +// // expected output: 3,2|3,3|2,2|0,1 +// +// List expected1 = new ArrayList(); +// expected1.add(3); +// expected1.add(2); +// expected1.add(3); +// expected1.add(3); +// expected1.add(2); +// expected1.add(2); +// expected1.add(0); +// expected1.add(1); +// +// List actual1 = MockCoContext.createAndExecute(invokable1, input11, input12); +// assertEquals(expected1, actual1); +// +// CoStreamWindow, Tuple2, Integer> invokable2 = new CoStreamWindow, Tuple2, Integer>( +// new MyCoGroup2(), 2, 3, new TimestampWrapper>(new MyTS2(), +// 1), new TimestampWrapper>(new MyTS2(), 1)); +// +// // WindowSize 2, slide 3 +// // 1,2|4,5|7,8| +// +// List> input21 = new ArrayList>(); +// input21.add(new Tuple2(1, 1)); +// input21.add(new Tuple2(1, 2)); +// input21.add(new Tuple2(2, 3)); +// input21.add(new Tuple2(3, 4)); +// input21.add(new Tuple2(3, 5)); +// input21.add(new Tuple2(4, 6)); +// input21.add(new Tuple2(4, 7)); +// input21.add(new Tuple2(5, 8)); +// +// List> input22 = new ArrayList>(); +// input22.add(new Tuple2(1, 1)); +// input22.add(new Tuple2(2, 0)); +// input22.add(new Tuple2(2, 2)); +// input22.add(new Tuple2(3, 9)); +// input22.add(new Tuple2(3, 4)); +// input22.add(new Tuple2(4, 10)); +// input22.add(new Tuple2(5, 8)); +// input22.add(new Tuple2(5, 7)); +// +// List expected2 = new ArrayList(); +// expected2.add(1); +// expected2.add(2); +// expected2.add(8); +// expected2.add(7); +// +// List actual2 = MockCoContext.createAndExecute(invokable2, input21, input22); +// assertEquals(expected2, actual2); +// } +//} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java index c8b0ae32c38ea..f11189034849c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java @@ -25,10 +25,9 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.operators.windowing.ParallelGroupedMerge; -import org.apache.flink.streaming.api.operators.windowing.ParallelMerge; +import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class ParallelMergeTest { @@ -45,37 +44,38 @@ public Integer reduce(Integer a, Integer b) throws Exception { } }; - TestCollector> out = new TestCollector>(); - List> output = out.getCollected(); + TestOutput> output = new TestOutput>(); + TimestampedCollector> collector = new TimestampedCollector>(output); + List> result = output.getCollected(); ParallelMerge merger = new ParallelMerge(reducer); merger.numberOfDiscretizers = 2; - merger.flatMap1(createTestWindow(1), out); - merger.flatMap1(createTestWindow(1), out); - merger.flatMap2(new Tuple2(1, 1), out); - assertTrue(output.isEmpty()); - merger.flatMap2(new Tuple2(1, 1), out); - assertEquals(StreamWindow.fromElements(2), output.get(0)); - - merger.flatMap2(new Tuple2(2, 2), out); - merger.flatMap1(createTestWindow(2), out); - merger.flatMap1(createTestWindow(2), out); - merger.flatMap2(new Tuple2(2, 1), out); - assertEquals(1, output.size()); - merger.flatMap1(createTestWindow(2), out); - assertEquals(StreamWindow.fromElements(3), output.get(1)); + merger.flatMap1(createTestWindow(1), collector); + merger.flatMap1(createTestWindow(1), collector); + merger.flatMap2(new Tuple2(1, 1), collector); + assertTrue(result.isEmpty()); + merger.flatMap2(new Tuple2(1, 1), collector); + assertEquals(StreamWindow.fromElements(2), result.get(0)); + + merger.flatMap2(new Tuple2(2, 2), collector); + merger.flatMap1(createTestWindow(2), collector); + merger.flatMap1(createTestWindow(2), collector); + merger.flatMap2(new Tuple2(2, 1), collector); + assertEquals(1, result.size()); + merger.flatMap1(createTestWindow(2), collector); + assertEquals(StreamWindow.fromElements(3), result.get(1)); // check error handling - merger.flatMap1(createTestWindow(3), out); - merger.flatMap2(new Tuple2(3, 1), out); - merger.flatMap2(new Tuple2(3, 1), out); + merger.flatMap1(createTestWindow(3), collector); + merger.flatMap2(new Tuple2(3, 1), collector); + merger.flatMap2(new Tuple2(3, 1), collector); - merger.flatMap2(new Tuple2(4, 1), out); - merger.flatMap2(new Tuple2(4, 1), out); - merger.flatMap1(createTestWindow(4), out); + merger.flatMap2(new Tuple2(4, 1), collector); + merger.flatMap2(new Tuple2(4, 1), collector); + merger.flatMap1(createTestWindow(4), collector); try { - merger.flatMap1(createTestWindow(4), out); + merger.flatMap1(createTestWindow(4), collector); fail(); } catch (RuntimeException e) { // Do nothing @@ -83,12 +83,12 @@ public Integer reduce(Integer a, Integer b) throws Exception { ParallelMerge merger2 = new ParallelMerge(reducer); merger2.numberOfDiscretizers = 2; - merger2.flatMap1(createTestWindow(0), out); - merger2.flatMap1(createTestWindow(1), out); - merger2.flatMap1(createTestWindow(1), out); - merger2.flatMap2(new Tuple2(1, 1), out); + merger2.flatMap1(createTestWindow(0), collector); + merger2.flatMap1(createTestWindow(1), collector); + merger2.flatMap1(createTestWindow(1), collector); + merger2.flatMap2(new Tuple2(1, 1), collector); try { - merger2.flatMap2(new Tuple2(1, 1), out); + merger2.flatMap2(new Tuple2(1, 1), collector); fail(); } catch (RuntimeException e) { // Do nothing @@ -99,18 +99,19 @@ public Integer reduce(Integer a, Integer b) throws Exception { @Test public void groupedTest() throws Exception { - TestCollector> out = new TestCollector>(); - List> output = out.getCollected(); + TestOutput> output = new TestOutput>(); + TimestampedCollector> collector = new TimestampedCollector>(output); + List> result = output.getCollected(); ParallelMerge merger = new ParallelGroupedMerge(); merger.numberOfDiscretizers = 2; - merger.flatMap1(createTestWindow(1), out); - merger.flatMap1(createTestWindow(1), out); - merger.flatMap2(new Tuple2(1, 1), out); - assertTrue(output.isEmpty()); - merger.flatMap2(new Tuple2(1, 1), out); - assertEquals(StreamWindow.fromElements(1, 1), output.get(0)); + merger.flatMap1(createTestWindow(1), collector); + merger.flatMap1(createTestWindow(1), collector); + merger.flatMap2(new Tuple2(1, 1), collector); + assertTrue(result.isEmpty()); + merger.flatMap2(new Tuple2(1, 1), collector); + assertEquals(StreamWindow.fromElements(1, 1), result.get(0)); } private StreamWindow createTestWindow(Integer id) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java similarity index 99% rename from flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowIntegrationTest.java rename to flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java index ec8cda82be990..5e6ffa28fb882 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowIntegrationTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java @@ -44,7 +44,7 @@ import org.apache.flink.util.Collector; import org.junit.Test; -public class WindowIntegrationTest implements Serializable { +public class WindowingITCase implements Serializable { private static final long serialVersionUID = 1L; private static final Integer MEMORYSIZE = 32; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java index eb49e2688e09a..6e220218e6863 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java @@ -48,6 +48,8 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.InstantiationUtil; @@ -103,12 +105,13 @@ public void simpleStateTest() throws Exception { @Test public void apiTest() throws Exception { StreamExecutionEnvironment env = new TestStreamEnvironment(3, 32); - + KeyedDataStream keyedStream = env.fromCollection(Arrays.asList(0, 1, 2, 3, 4, 5, 6)).keyBy(new ModKey(4)); keyedStream.map(new StatefulMapper()).addSink(new SinkFunction() { private static final long serialVersionUID = 1L; - public void invoke(String value) throws Exception {} + public void invoke(String value) throws Exception { + } }); keyedStream.map(new StatefulMapper2()).setParallelism(1).addSink(new SinkFunction() { @@ -128,8 +131,8 @@ public void invoke(String value) throws Exception {} private void processInputs(StreamMap map, List input) throws Exception { for (Integer i : input) { - map.getRuntimeContext().setNextInput(i); - map.processElement(i); + map.getRuntimeContext().setNextInput(new StreamRecord(i, 0L)); + map.processElement(new StreamRecord(i, 0L)); } } @@ -144,11 +147,16 @@ private StreamMap createOperatorWithContext(List output StreamMap op = new StreamMap(new StatefulMapper()); - op.setup(new Output() { + op.setup(new Output>() { @Override - public void collect(String record) { - outputList.add(record); + public void collect(StreamRecord record) { + outputList.add(record.getValue()); + } + + @Override + public void emitWatermark(Watermark mark) { + } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java index 4ac7fda84b2bf..317a21c023247 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java @@ -40,6 +40,6 @@ public boolean initList() { @Override public void emit(SerializationDelegate>> record) { - emittedRecords.add(record.getInstance().getObject().f0); + emittedRecords.add(record.getInstance().getValue().f0); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java index 967c719cd7157..6bc0e30fd021b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java @@ -22,10 +22,10 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBuffer; -import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer; -import org.apache.flink.util.Collector; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.junit.Test; public class BasicWindowBufferTest { @@ -33,7 +33,7 @@ public class BasicWindowBufferTest { @Test public void testEmitWindow() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); List> collected = collector.getCollected(); WindowBuffer wb = new BasicWindowBuffer(); @@ -60,13 +60,13 @@ public void testEmitWindow() throws Exception { assertEquals(2, collected.size()); } - public static class TestCollector implements Collector { + public static class TestOutput implements Output> { private final List collected = new ArrayList(); @Override - public void collect(T record) { - collected.add(record); + public void collect(StreamRecord record) { + collected.add(record.getValue()); } @Override @@ -77,6 +77,10 @@ public List getCollected() { return collected; } + @Override + public void emitWatermark(Watermark mark) { + + } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java index c91910b1b1b8f..84304998d54ba 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java @@ -32,8 +32,9 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; import org.apache.flink.streaming.util.keys.KeySelectorUtil; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; + import org.junit.Test; public class JumpingCountGroupedPreReducerTest { @@ -58,7 +59,7 @@ public void testEmitWindow() throws Exception { inputs.add(new Tuple2(1, -2)); inputs.add(new Tuple2(100, -200)); - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); List>> collected = collector.getCollected(); WindowBuffer> wb = new JumpingCountGroupedPreReducer>( @@ -109,7 +110,7 @@ public void testEmitWindow2() throws Exception { inputs.add(new Tuple2(1, -2)); inputs.add(new Tuple2(100, -200)); - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); List>> collected = collector.getCollected(); WindowBuffer> wb = new JumpingCountGroupedPreReducer>( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.java index ba890ab81d31f..2279264243ef5 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.java @@ -27,7 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class JumpingCountPreReducerTest { @@ -48,7 +48,7 @@ public void testEmitWindow() throws Exception { inputs.add(new Tuple2(4, -2)); inputs.add(new Tuple2(5, -3)); - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); List>> collected = collector.getCollected(); WindowBuffer> wb = new JumpingCountPreReducer>( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java index 5b693e72b5b24..ce312d34edb02 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java @@ -27,7 +27,7 @@ import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.helper.Timestamp; import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class JumpingTimePreReducerTest { @@ -39,7 +39,7 @@ public class JumpingTimePreReducerTest { @Test public void testEmitWindow() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); List> collected = collector.getCollected(); WindowBuffer wb = new JumpingTimePreReducer( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java index 377bdb5a16b3d..7f585278db771 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java @@ -1,34 +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. - */ +* 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.windowing.windowbuffer; -import static org.apache.flink.streaming.api.windowing.windowbuffer.SlidingTimeGroupedPreReducerTest.checkResults; +import static org.junit.Assert.assertEquals; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.streaming.api.operators.windowing.WindowIntegrationTest; +import org.apache.flink.streaming.api.operators.windowing.WindowingITCase; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class SlidingCountGroupedPreReducerTest { @@ -37,11 +38,11 @@ public class SlidingCountGroupedPreReducerTest { ReduceFunction reducer = new SumReducer(); - KeySelector key = new WindowIntegrationTest.ModKey(2); + KeySelector key = new WindowingITCase.ModKey(2); @Test public void testPreReduce1() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountGroupedPreReducer preReducer = new SlidingCountGroupedPreReducer( reducer, serializer, key, 3, 2, 0); @@ -84,7 +85,7 @@ public void testPreReduce1() throws Exception { @Test public void testPreReduce2() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountGroupedPreReducer preReducer = new SlidingCountGroupedPreReducer( reducer, serializer, key, 5, 2, 0); @@ -126,7 +127,7 @@ public void testPreReduce2() throws Exception { @Test public void testPreReduce3() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountGroupedPreReducer preReducer = new SlidingCountGroupedPreReducer( reducer, serializer, key, 6, 3, 0); @@ -163,7 +164,7 @@ public void testPreReduce3() throws Exception { @Test public void testPreReduce4() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountGroupedPreReducer preReducer = new SlidingCountGroupedPreReducer( reducer, serializer, key, 5, 1, 2); @@ -217,4 +218,18 @@ public Integer reduce(Integer value1, Integer value2) throws Exception { } + + protected static void checkResults(List> expected, + List> actual) { + + for (StreamWindow sw : expected) { + Collections.sort(sw); + } + + for (StreamWindow sw : actual) { + Collections.sort(sw); + } + + assertEquals(expected, actual); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java index 3ce65f1185bb2..156b8750ee69d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java @@ -26,7 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class SlidingCountPreReducerTest { @@ -37,7 +37,7 @@ public class SlidingCountPreReducerTest { @Test public void testPreReduce1() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountPreReducer preReducer = new SlidingCountPreReducer(reducer, serializer, 3, 2, 0); @@ -80,7 +80,7 @@ public void testPreReduce1() throws Exception { @Test public void testPreReduce2() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountPreReducer preReducer = new SlidingCountPreReducer(reducer, serializer, 5, 2, 0); @@ -122,7 +122,7 @@ public void testPreReduce2() throws Exception { @Test public void testPreReduce3() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountPreReducer preReducer = new SlidingCountPreReducer(reducer, serializer, 6, 3, 0); @@ -159,7 +159,7 @@ public void testPreReduce3() throws Exception { @Test public void testPreReduce4() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingCountPreReducer preReducer = new SlidingCountPreReducer(reducer, serializer, 5, 1, 2); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java index 3f1cba148d8b0..68bceda5c11eb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java @@ -31,11 +31,11 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.typeutils.TypeInfoParser; -import org.apache.flink.streaming.api.operators.windowing.WindowIntegrationTest; +import org.apache.flink.streaming.api.operators.windowing.WindowingITCase; import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.helper.Timestamp; import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class SlidingTimeGroupedPreReducerTest { @@ -48,7 +48,7 @@ public class SlidingTimeGroupedPreReducerTest { ReduceFunction> tupleReducer = new TupleSumReducer(); - KeySelector key = new WindowIntegrationTest.ModKey(2); + KeySelector key = new WindowingITCase.ModKey(2); KeySelector, ?> tupleKey = new TupleModKey(2); @Test @@ -58,7 +58,7 @@ public void testPreReduce1() throws Exception { // replaying the same sequence of elements with a later timestamp and expecting the same // result. - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); SlidingTimeGroupedPreReducer> preReducer = new SlidingTimeGroupedPreReducer>(tupleReducer, tupleType.createSerializer(new ExecutionConfig()), tupleKey, 3, 2, new TimestampWrapper>(new Timestamp>() { @@ -190,7 +190,7 @@ protected static void checkResults(List> expected, @Test public void testPreReduce2() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingTimeGroupedPreReducer preReducer = new SlidingTimeGroupedPreReducer( reducer, serializer, key, 5, 2, new TimestampWrapper( @@ -241,7 +241,7 @@ public long getTimestamp(Integer value) { @Test public void testPreReduce3() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingTimeGroupedPreReducer preReducer = new SlidingTimeGroupedPreReducer( reducer, serializer, key, 6, 3, new TimestampWrapper( @@ -287,7 +287,7 @@ public long getTimestamp(Integer value) { @Test public void testPreReduce4() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingTimeGroupedPreReducer preReducer = new SlidingTimeGroupedPreReducer( reducer, serializer, key, 3, 2, new TimestampWrapper( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java index 0519da7f5f752..6a36c57623b7a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java @@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.windowing.StreamWindow; import org.apache.flink.streaming.api.windowing.helper.Timestamp; import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class SlidingTimePreReducerTest { @@ -50,7 +50,7 @@ public void testPreReduce1() throws Exception { // replaying the same sequence of elements with a later timestamp and expecting the same // result. - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); SlidingTimePreReducer> preReducer = new SlidingTimePreReducer>(tupleReducer, tupleType.createSerializer(new ExecutionConfig()), 3, 2, new TimestampWrapper>(new Timestamp>() { @@ -145,7 +145,7 @@ public long getTimestamp(Tuple2 value) { @Test public void testPreReduce2() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingTimePreReducer preReducer = new SlidingTimePreReducer(reducer, serializer, 5, 2, new TimestampWrapper(new Timestamp() { @@ -195,7 +195,7 @@ public long getTimestamp(Integer value) { @Test public void testPreReduce3() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingTimePreReducer preReducer = new SlidingTimePreReducer(reducer, serializer, 6, 3, new TimestampWrapper(new Timestamp() { @@ -240,7 +240,7 @@ public long getTimestamp(Integer value) { @Test public void testPreReduce4() throws Exception { - TestCollector> collector = new TestCollector>(); + TestOutput> collector = new TestOutput>(); SlidingTimePreReducer preReducer = new SlidingTimePreReducer(reducer, serializer, 3, 2, new TimestampWrapper(new Timestamp() { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java index c5107bfe01860..3aee2881b1509 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java @@ -32,7 +32,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.apache.flink.streaming.util.keys.KeySelectorUtil; import org.junit.Test; @@ -57,7 +57,7 @@ public void testEmitWindow() throws Exception { inputs.add(new Tuple2(1, -1)); inputs.add(new Tuple2(1, -2)); - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); List>> collected = collector.getCollected(); WindowBuffer> wb = new TumblingGroupedPreReducer>( @@ -104,7 +104,7 @@ public void testEmitWindow2() throws Exception { inputs.add(new Tuple2(1, -1)); inputs.add(new Tuple2(1, -2)); - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); List>> collected = collector.getCollected(); WindowBuffer> wb = new TumblingGroupedPreReducer>( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java index b8de02e8d1d91..3e537a5c512cb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java @@ -27,9 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.windowing.StreamWindow; -import org.apache.flink.streaming.api.windowing.windowbuffer.TumblingPreReducer; -import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer; -import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestCollector; +import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput; import org.junit.Test; public class TumblingPreReducerTest { @@ -49,7 +47,7 @@ public void testEmitWindow() throws Exception { inputs.add(new Tuple2(3, -1)); inputs.add(new Tuple2(4, -2)); - TestCollector>> collector = new TestCollector>>(); + TestOutput>> collector = new TestOutput>>(); List>> collected = collector.getCollected(); WindowBuffer> wb = new TumblingPreReducer>( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferIOTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferIOTest.java index 3f8401d3a81c1..d8a36967fbcd0 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferIOTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferIOTest.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.io.BarrierBuffer; import org.junit.Test; public class BarrierBufferIOTest { @@ -55,7 +54,7 @@ public void IOTest() throws IOException, InterruptedException { if (boe.isBuffer()) { boe.getBuffer().recycle(); } else { - barrierBuffer.processSuperstep(boe); + barrierBuffer.processBarrier(boe); } } // System.out.println("Ran for " + (System.currentTimeMillis() - @@ -101,14 +100,14 @@ protected static class MockInputGate implements InputGate { private int numChannels; private BufferPool[] bufferPools; - private int[] currentSupersteps; + private int[] currentBarriers; BarrierGenerator[] barrierGens; int currentChannel = 0; long c = 0; public MockInputGate(BufferPool[] bufferPools, BarrierGenerator[] barrierGens) { this.numChannels = bufferPools.length; - this.currentSupersteps = new int[numChannels]; + this.currentBarriers = new int[numChannels]; this.bufferPools = bufferPools; this.barrierGens = barrierGens; } @@ -132,7 +131,7 @@ public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedExcep currentChannel = (currentChannel + 1) % numChannels; if (barrierGens[currentChannel].isNextBarrier()) { - return BarrierBufferTest.createSuperstep(++currentSupersteps[currentChannel], + return BarrierBufferTest.createBarrier(++currentBarriers[currentChannel], currentChannel); } else { Buffer buffer = bufferPools[currentChannel].requestBuffer(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index 89ec7dc09c95a..cb5e046e49f90 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.streaming.runtime.tasks.StreamingSuperstep; +import org.apache.flink.streaming.runtime.tasks.CheckpointBarrier; import org.junit.Test; @@ -67,10 +67,10 @@ public void testOneChannelBarrier() throws IOException, InterruptedException { List input = new LinkedList(); input.add(createBuffer(0)); input.add(createBuffer(0)); - input.add(createSuperstep(1, 0)); + input.add(createBarrier(1, 0)); input.add(createBuffer(0)); input.add(createBuffer(0)); - input.add(createSuperstep(2, 0)); + input.add(createBarrier(2, 0)); input.add(createBuffer(0)); InputGate mockIG = new MockInputGate(1, input); @@ -82,11 +82,11 @@ public void testOneChannelBarrier() throws IOException, InterruptedException { assertEquals(input.get(0), nextBoe = bb.getNextNonBlocked()); assertEquals(input.get(1), nextBoe = bb.getNextNonBlocked()); assertEquals(input.get(2), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); assertEquals(input.get(3), nextBoe = bb.getNextNonBlocked()); assertEquals(input.get(4), nextBoe = bb.getNextNonBlocked()); assertEquals(input.get(5), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); assertEquals(input.get(6), nextBoe = bb.getNextNonBlocked()); bb.cleanup(); @@ -98,18 +98,18 @@ public void testMultiChannelBarrier() throws IOException, InterruptedException { List input = new LinkedList(); input.add(createBuffer(0)); input.add(createBuffer(1)); - input.add(createSuperstep(1, 0)); - input.add(createSuperstep(2, 0)); + input.add(createBarrier(1, 0)); + input.add(createBarrier(2, 0)); input.add(createBuffer(0)); - input.add(createSuperstep(3, 0)); + input.add(createBarrier(3, 0)); input.add(createBuffer(0)); input.add(createBuffer(1)); - input.add(createSuperstep(1, 1)); + input.add(createBarrier(1, 1)); input.add(createBuffer(0)); input.add(createBuffer(1)); - input.add(createSuperstep(2, 1)); - input.add(createSuperstep(3, 1)); - input.add(createSuperstep(4, 0)); + input.add(createBarrier(2, 1)); + input.add(createBarrier(3, 1)); + input.add(createBarrier(4, 0)); input.add(createBuffer(0)); input.add(new BufferOrEvent(new EndOfPartitionEvent(), 1)); @@ -123,24 +123,24 @@ public void testMultiChannelBarrier() throws IOException, InterruptedException { check(input.get(0), nextBoe = bb.getNextNonBlocked()); check(input.get(1), nextBoe = bb.getNextNonBlocked()); check(input.get(2), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); check(input.get(7), nextBoe = bb.getNextNonBlocked()); check(input.get(8), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); check(input.get(3), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); check(input.get(10), nextBoe = bb.getNextNonBlocked()); check(input.get(11), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); check(input.get(4), nextBoe = bb.getNextNonBlocked()); check(input.get(5), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); check(input.get(12), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); check(input.get(6), nextBoe = bb.getNextNonBlocked()); check(input.get(9), nextBoe = bb.getNextNonBlocked()); check(input.get(13), nextBoe = bb.getNextNonBlocked()); - bb.processSuperstep(nextBoe); + bb.processBarrier(nextBoe); check(input.get(14), nextBoe = bb.getNextNonBlocked()); check(input.get(15), nextBoe = bb.getNextNonBlocked()); @@ -206,8 +206,8 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { } } - protected static BufferOrEvent createSuperstep(long id, int channel) { - return new BufferOrEvent(new StreamingSuperstep(id, System.currentTimeMillis()), channel); + protected static BufferOrEvent createBarrier(long id, int channel) { + return new BufferOrEvent(new CheckpointBarrier(id, System.currentTimeMillis()), channel); } protected static BufferOrEvent createBuffer(int channel) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/CoRecordReaderTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/CoRecordReaderTest.java deleted file mode 100644 index 528829d43a9f5..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/CoRecordReaderTest.java +++ /dev/null @@ -1,92 +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.runtime.io; - -import static org.junit.Assert.assertEquals; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.streaming.runtime.io.BarrierBuffer; -import org.apache.flink.streaming.runtime.io.CoRecordReader; -import org.apache.flink.streaming.runtime.io.BarrierBufferTest.MockInputGate; -import org.junit.Test; - -public class CoRecordReaderTest { - - @Test - public void test() throws InterruptedException, IOException { - - List input1 = new LinkedList(); - input1.add(BarrierBufferTest.createBuffer(0)); - input1.add(BarrierBufferTest.createSuperstep(1, 0)); - input1.add(BarrierBufferTest.createBuffer(0)); - - InputGate ig1 = new MockInputGate(1, input1); - - List input2 = new LinkedList(); - input2.add(BarrierBufferTest.createBuffer(0)); - input2.add(BarrierBufferTest.createBuffer(0)); - input2.add(BarrierBufferTest.createSuperstep(1, 0)); - input2.add(BarrierBufferTest.createBuffer(0)); - - InputGate ig2 = new MockInputGate(1, input2); - - CoRecordReader coReader = new CoRecordReader( - ig1, ig2); - BarrierBuffer b1 = coReader.barrierBuffer1; - BarrierBuffer b2 = coReader.barrierBuffer2; - - coReader.addToAvailable(ig1); - coReader.addToAvailable(ig2); - coReader.addToAvailable(ig2); - coReader.addToAvailable(ig1); - - assertEquals(1, coReader.getNextReaderIndexBlocking()); - b1.getNextNonBlocked(); - - assertEquals(2, coReader.getNextReaderIndexBlocking()); - b2.getNextNonBlocked(); - - assertEquals(2, coReader.getNextReaderIndexBlocking()); - b2.getNextNonBlocked(); - - assertEquals(1, coReader.getNextReaderIndexBlocking()); - b1.getNextNonBlocked(); - b1.processSuperstep(input1.get(1)); - - coReader.addToAvailable(ig1); - coReader.addToAvailable(ig2); - coReader.addToAvailable(ig2); - - assertEquals(2, coReader.getNextReaderIndexBlocking()); - b2.getNextNonBlocked(); - b2.processSuperstep(input2.get(2)); - - assertEquals(1, coReader.getNextReaderIndexBlocking()); - b1.getNextNonBlocked(); - - assertEquals(2, coReader.getNextReaderIndexBlocking()); - b2.getNextNonBlocked(); - } - -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java index aa4d24a9fa8c0..a1cea1303ec9a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.junit.Before; import org.junit.Test; @@ -32,7 +31,7 @@ public class BroadcastPartitionerTest { private BroadcastPartitioner broadcastPartitioner2; private BroadcastPartitioner broadcastPartitioner3; - private StreamRecord streamRecord = new StreamRecord(); + private StreamRecord streamRecord = new StreamRecord(null); private SerializationDelegate> sd = new SerializationDelegate>(null); @Before diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java index b37e43a2e3389..2643bbac11a09 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java @@ -28,7 +28,7 @@ public class DistributePartitionerTest { private RebalancePartitioner distributePartitioner; - private StreamRecord streamRecord = new StreamRecord(); + private StreamRecord streamRecord = new StreamRecord(null); private SerializationDelegate> sd = new SerializationDelegate>( null); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java index 94d29accae958..05541f5eec2c6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java @@ -21,34 +21,28 @@ import static org.junit.Assert.assertEquals; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.runtime.partitioner.FieldsPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.junit.Before; import org.junit.Test; public class FieldsPartitionerTest { - private FieldsPartitioner fieldsPartitioner; - private StreamRecord streamRecord1 = new StreamRecord() - .setObject(new Tuple2("test", 0)); - private StreamRecord streamRecord2 = new StreamRecord() - .setObject(new Tuple2("test", 42)); - private SerializationDelegate> sd1 = new SerializationDelegate>( - null); - private SerializationDelegate> sd2 = new SerializationDelegate>( - null); + private FieldsPartitioner> fieldsPartitioner; + private StreamRecord> streamRecord1 = new StreamRecord>(new Tuple2("test", 0)); + private StreamRecord> streamRecord2 = new StreamRecord>(new Tuple2("test", 42)); + private SerializationDelegate>> sd1 = new SerializationDelegate>>(null); + private SerializationDelegate>> sd2 = new SerializationDelegate>>(null); @Before public void setPartitioner() { - fieldsPartitioner = new FieldsPartitioner(new KeySelector() { + fieldsPartitioner = new FieldsPartitioner>(new KeySelector, String>() { private static final long serialVersionUID = 1L; @Override - public String getKey(Tuple value) throws Exception { + public String getKey(Tuple2 value) throws Exception { return value.getField(0); } }); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java index 405a28e132f73..856f7aa78d603 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java @@ -28,7 +28,7 @@ public class ForwardPartitionerTest { private RebalancePartitioner forwardPartitioner; - private StreamRecord streamRecord = new StreamRecord(); + private StreamRecord streamRecord = new StreamRecord(null); private SerializationDelegate> sd = new SerializationDelegate>( null); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java index 69c00cda67cb5..6ae37305317d1 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.junit.Before; import org.junit.Test; @@ -29,7 +28,7 @@ public class GlobalPartitionerTest { private GlobalPartitioner globalPartitioner; - private StreamRecord streamRecord = new StreamRecord(); + private StreamRecord streamRecord = new StreamRecord(null); private SerializationDelegate> sd = new SerializationDelegate>( null); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java index d99a21e013a5b..aff177c1e301b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java @@ -22,7 +22,6 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.junit.Before; import org.junit.Test; @@ -30,7 +29,7 @@ public class ShufflePartitionerTest { private ShufflePartitioner shufflePartitioner; - private StreamRecord streamRecord = new StreamRecord(); + private StreamRecord streamRecord = new StreamRecord(null); private SerializationDelegate> sd = new SerializationDelegate>( null); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java new file mode 100644 index 0000000000000..d623dd85be4e8 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -0,0 +1,339 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.runtime.tasks; + + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * Tests for {@link OneInputStreamTask}. + * + *

+ * Note:
+ * We only use a {@link StreamMap} operator here. We also test the individual operators but Map is + * used as a representative to test OneInputStreamTask, since OneInputStreamTask is used for all + * OneInputStreamOperators. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ResultPartitionWriter.class}) +public class OneInputStreamTaskTest { + + /** + * This test verifies that open() and close() are correctly called. This test also verifies + * that timestamps of emitted elements are correct. {@link StreamMap} assigns the input + * timestamp to emitted elements. + */ + @Test + public void testOpenCloseAndTimestamps() throws Exception { + final OneInputStreamTask mapTask = new OneInputStreamTask(); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + StreamMap mapOperator = new StreamMap(new TestOpenCloseMapFunction()); + streamConfig.setStreamOperator(mapOperator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.invoke(); + + testHarness.processElement(new StreamRecord("Hello", initialTime + 1)); + testHarness.processElement(new StreamRecord("Ciao", initialTime + 2)); + expectedOutput.add(new StreamRecord("Hello", initialTime + 1)); + expectedOutput.add(new StreamRecord("Ciao", initialTime + 2)); + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseMapFunction.closeCalled); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", + expectedOutput, + testHarness.getOutput()); + } + + /** + * This test verifies that watermarks are correctly forwarded. This also checks whether + * watermarks are forwarded only when we have received watermarks from all inputs. The + * forwarded watermark must be the minimum of the watermarks of all inputs. + */ + @Test + @SuppressWarnings("unchecked") + public void testWatermarkForwarding() throws Exception { + final OneInputStreamTask mapTask = new OneInputStreamTask(); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + StreamMap mapOperator = new StreamMap(new IdentityMap()); + streamConfig.setStreamOperator(mapOperator); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + long initialTime = 0L; + + testHarness.invoke(); + + testHarness.processElement(new Watermark(initialTime), 0, 0); + testHarness.processElement(new Watermark(initialTime), 0, 1); + testHarness.processElement(new Watermark(initialTime), 1, 0); + + // now the output should still be empty + testHarness.waitForInputProcessing(); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.processElement(new Watermark(initialTime), 1, 1); + + // now the watermark should have propagated, Map simply forward Watermarks + testHarness.waitForInputProcessing(); + expectedOutput.add(new Watermark(initialTime)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", + expectedOutput, + testHarness.getOutput()); + + // contrary to checkpoint barriers these elements are not blocked by watermarks + testHarness.processElement(new StreamRecord("Hello", initialTime)); + testHarness.processElement(new StreamRecord("Ciao", initialTime)); + expectedOutput.add(new StreamRecord("Hello", initialTime)); + expectedOutput.add(new StreamRecord("Ciao", initialTime)); + + testHarness.processElement(new Watermark(initialTime + 4), 0, 0); + testHarness.processElement(new Watermark(initialTime + 3), 0, 1); + testHarness.processElement(new Watermark(initialTime + 3), 1, 0); + testHarness.processElement(new Watermark(initialTime + 2), 1, 1); + + // check whether we get the minimum of all the watermarks, this must also only occur in + // the output after the two StreamRecords + testHarness.waitForInputProcessing(); + expectedOutput.add(new Watermark(initialTime + 2)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + + // advance watermark from one of the inputs, now we should get a now one since the + // minimum increases + testHarness.processElement(new Watermark(initialTime + 4), 1, 1); + testHarness.waitForInputProcessing(); + expectedOutput.add(new Watermark(initialTime + 3)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + // advance the other two inputs, now we should get a new one since the + // minimum increases again + testHarness.processElement(new Watermark(initialTime + 4), 0, 1); + testHarness.processElement(new Watermark(initialTime + 4), 1, 0); + testHarness.waitForInputProcessing(); + expectedOutput.add(new Watermark(initialTime + 4)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + List resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); + Assert.assertEquals(2, resultElements.size()); + } + + /** + * This test verifies that checkpoint barriers are correctly forwarded. + */ + @Test + @SuppressWarnings("unchecked") + public void testCheckpointBarriers() throws Exception { + final OneInputStreamTask mapTask = new OneInputStreamTask(); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + StreamMap mapOperator = new StreamMap(new IdentityMap()); + streamConfig.setStreamOperator(mapOperator); + + Queue expectedOutput = new ConcurrentLinkedQueue(); + long initialTime = 0L; + + testHarness.invoke(); + + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0); + + // These elements should be buffered until we receive barriers from + // all inputs + testHarness.processElement(new StreamRecord("Hello-0-0", initialTime), 0, 0); + testHarness.processElement(new StreamRecord("Ciao-0-0", initialTime), 0, 0); + + // These elements should be forwarded, since we did not yet receive a checkpoint barrier + // on that input, only add to same input, otherwise we would not know the ordering + // of the output since the Task might read the inputs in any order + testHarness.processElement(new StreamRecord("Hello-1-1", initialTime), 1, 1); + testHarness.processElement(new StreamRecord("Ciao-1-1", initialTime), 1, 1); + expectedOutput.add(new StreamRecord("Hello-1-1", initialTime)); + expectedOutput.add(new StreamRecord("Ciao-1-1", initialTime)); + + testHarness.waitForInputProcessing(); + // we should not yet see the barrier, only the two elements from non-blocked input + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1); + + testHarness.waitForInputProcessing(); + + // now we should see the barrier and after that the buffered elements + expectedOutput.add(new CheckpointBarrier(0, 0)); + expectedOutput.add(new StreamRecord("Hello-0-0", initialTime)); + expectedOutput.add(new StreamRecord("Ciao-0-0", initialTime)); + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + /** + * This test verifies that checkpoint barriers and barrier buffers work correctly with + * concurrent checkpoint barriers where one checkpoint is "overtaking" another checkpoint, i.e. + * some inputs receive barriers from an earlier checkpoint, thereby blocking, + * then all inputs receive barriers from a later checkpoint. + */ + @Test + @SuppressWarnings("unchecked") + public void testOvertakingCheckpointBarriers() throws Exception { + final OneInputStreamTask mapTask = new OneInputStreamTask(); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + StreamMap mapOperator = new StreamMap(new IdentityMap()); + streamConfig.setStreamOperator(mapOperator); + + Queue expectedOutput = new ConcurrentLinkedQueue(); + long initialTime = 0L; + + testHarness.invoke(); + + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0); + + // These elements should be buffered until we receive barriers from + // all inputs + testHarness.processElement(new StreamRecord("Hello-0-0", initialTime), 0, 0); + testHarness.processElement(new StreamRecord("Ciao-0-0", initialTime), 0, 0); + + // These elements should be forwarded, since we did not yet receive a checkpoint barrier + // on that input, only add to same input, otherwise we would not know the ordering + // of the output since the Task might read the inputs in any order + testHarness.processElement(new StreamRecord("Hello-1-1", initialTime), 1, 1); + testHarness.processElement(new StreamRecord("Ciao-1-1", initialTime), 1, 1); + expectedOutput.add(new StreamRecord("Hello-1-1", initialTime)); + expectedOutput.add(new StreamRecord("Ciao-1-1", initialTime)); + + testHarness.waitForInputProcessing(); + // we should not yet see the barrier, only the two elements from non-blocked input + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + // Now give a later barrier to all inputs, this should unblock the first channel, + // thereby allowing the two blocked elements through + testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 0); + testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 1); + testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 0); + testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 1); + + expectedOutput.add(new StreamRecord("Hello-0-0", initialTime)); + expectedOutput.add(new StreamRecord("Ciao-0-0", initialTime)); + expectedOutput.add(new CheckpointBarrier(1, 1)); + + testHarness.waitForInputProcessing(); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + + // Then give the earlier barrier, these should be ignored + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1); + + testHarness.waitForInputProcessing(); + + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseMapFunction extends RichMapFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public String map(String value) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return value; + } + } + + private static class IdentityMap implements MapFunction { + private static final long serialVersionUID = 1L; + + @Override + public String map(String value) throws Exception { + return value; + } + } +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java new file mode 100644 index 0000000000000..a8029e6a2a38c --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.runtime.tasks; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; + +import java.io.IOException; + + +/** + * Test harness for testing a {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}. + * + *

+ * This mock Invokable provides the task with a basic runtime context and allows pushing elements + * and watermarks into the task. {@link #getOutput()} can be used to get the emitted elements + * and events. You are free to modify the retrieved list. + * + *

+ * After setting up everything the Task can be invoked using {@link #invoke()}. This will start + * a new Thread to execute the Task. Use {@link #waitForTaskCompletion()} to wait for the Task + * thread to finish. Use {@link #processElement} to send elements to the task. Use + * {@link #processEvent(AbstractEvent)} to send events to the task. + * Before waiting for the task to finish you must call {@link #endInput()} to signal to the task + * that data entry is finished. + * + *

+ * When Elements or Events are offered to the Task they are put into a queue. The input gates + * of the Task read from this queue. Use {@link #waitForInputProcessing()} to wait until all + * queues are empty. This must be used after entering some elements before checking the + * desired output. + * + *

+ * When using this you need to add the following line to your test class to setup Powermock: + * {@code @PrepareForTest({ResultPartitionWriter.class})} + */ +public class OneInputStreamTaskTestHarness extends StreamTaskTestHarness { + + private TypeInformation inputType; + private TypeSerializer inputSerializer; + + /** + * Creates a test harness with the specified number of input gates and specified number + * of channels per input gate. + */ + public OneInputStreamTaskTestHarness(OneInputStreamTask task, + int numInputGates, + + int numInputChannelsPerGate, + TypeInformation inputType, + TypeInformation outputType) { + super(task, outputType); + + this.inputType = inputType; + inputSerializer = inputType.createSerializer(executionConfig); + + this.numInputGates = numInputGates; + this.numInputChannelsPerGate = numInputChannelsPerGate; + } + + /** + * Creates a test harness with one input gate that has one input channel. + */ + public OneInputStreamTaskTestHarness(OneInputStreamTask task, + TypeInformation inputType, + TypeInformation outputType) { + this(task, 1, 1, inputType, outputType); + } + + @Override + protected void initializeInputs() throws IOException, InterruptedException { + inputGates = new StreamTestSingleInputGate[numInputGates]; + + for (int i = 0; i < numInputGates; i++) { + inputGates[i] = new StreamTestSingleInputGate( + numInputChannelsPerGate, + bufferSize, + inputSerializer); + this.mockEnv.addInputGate(inputGates[i].getInputGate()); + } + + + streamConfig.setNumberOfInputs(1); + streamConfig.setTypeSerializerIn1(inputSerializer); + } + +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index b4877c6ae05b8..f34eafe88a4c5 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -18,50 +18,63 @@ package org.apache.flink.streaming.runtime.tasks; -import java.io.IOException; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.state.OperatorState; -import org.apache.flink.api.common.state.StateCheckpointer; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.taskmanager.Task; -import org.apache.flink.streaming.api.collector.selector.BroadcastOutputSelectorWrapper; -import org.apache.flink.streaming.api.collector.selector.OutputSelector; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.api.graph.StreamEdge; -import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; -import org.apache.flink.util.StringUtils; +import org.apache.flink.streaming.util.TestHarnessUtil; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.io.Serializable; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicLong; + +/** + * These tests verify that the RichFunction methods are called (in correct order). And that + * checkpointing/element emission don't occur concurrently. + */ @RunWith(PowerMockRunner.class) -@PrepareForTest({Task.class, ResultPartitionWriter.class}) -public class SourceStreamTaskTest extends StreamTaskTestBase { +@PrepareForTest({ResultPartitionWriter.class}) +public class SourceStreamTaskTest { - private static final int MEMORY_MANAGER_SIZE = 1024 * 1024; - private static final int NETWORK_BUFFER_SIZE = 1024; + /** + * This test verifies that open() and close() are correctly called by the StreamTask. + */ + @Test + public void testOpenClose() throws Exception { + final SourceStreamTask sourceTask = new SourceStreamTask(); + final StreamTaskTestHarness testHarness = new StreamTaskTestHarness(sourceTask, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + StreamSource sourceOperator = new StreamSource(new OpenCloseTestSource()); + streamConfig.setStreamOperator(sourceOperator); + + testHarness.invoke(); + testHarness.waitForTaskCompletion(); + + Assert.assertTrue("RichFunction methods where not called.", OpenCloseTestSource.closeCalled); + + List resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); + Assert.assertEquals(10, resultElements.size()); + } /** * This test ensures that the SourceStreamTask properly serializes checkpointing @@ -76,7 +89,7 @@ public class SourceStreamTaskTest extends StreamTaskTestBase { * source kept emitting elements while the checkpoint was ongoing. */ @Test - public void testDataSourceTask() throws Exception { + public void testCheckpointing() throws Exception { final int NUM_ELEMENTS = 100; final int NUM_CHECKPOINTS = 100; final int NUM_CHECKPOINTERS = 1; @@ -84,38 +97,15 @@ public void testDataSourceTask() throws Exception { final int SOURCE_CHECKPOINT_DELAY = 1000; // how many random values we sum up in storeCheckpoint final int SOURCE_READ_DELAY = 1; // in ms - List> outList = new ArrayList>(); - - super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); - - StreamSource> sourceOperator = new StreamSource>(new MockSource(NUM_ELEMENTS, SOURCE_CHECKPOINT_DELAY, SOURCE_READ_DELAY)); + final TupleTypeInfo> typeInfo = new TupleTypeInfo>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); final SourceStreamTask> sourceTask = new SourceStreamTask>(); + final StreamTaskTestHarness> testHarness = new StreamTaskTestHarness>(sourceTask, typeInfo); - TupleTypeInfo> typeInfo = new TupleTypeInfo>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); - StreamRecordSerializer> streamSerializer = new StreamRecordSerializer>(typeInfo, new ExecutionConfig()); - - super.addOutput(outList, serializer); - - StreamConfig streamConfig = super.getStreamConfig(); - + StreamConfig streamConfig = testHarness.getStreamConfig(); + StreamSource> sourceOperator = new StreamSource>(new MockSource(NUM_ELEMENTS, SOURCE_CHECKPOINT_DELAY, SOURCE_READ_DELAY)); streamConfig.setStreamOperator(sourceOperator); - streamConfig.setChainStart(); - streamConfig.setOutputSelectorWrapper(new BroadcastOutputSelectorWrapper()); - streamConfig.setNumberOfOutputs(1); - - List outEdgesInOrder = new LinkedList(); - StreamNode sourceVertex = new StreamNode(null, 0, sourceOperator, "source", new LinkedList>(), SourceStreamTask.class); - StreamNode targetVertexDummy = new StreamNode(null, 1, sourceOperator, "target dummy", new LinkedList>(), SourceStreamTask.class); - outEdgesInOrder.add(new StreamEdge(sourceVertex, targetVertexDummy, 0, new LinkedList(), new BroadcastPartitioner())); - streamConfig.setOutEdgesInOrder(outEdgesInOrder); - streamConfig.setNonChainedOutputs(outEdgesInOrder); - streamConfig.setTypeSerializerOut1(streamSerializer); - streamConfig.setVertexID(0); - - super.registerTask(sourceTask); ExecutorService executor = Executors.newFixedThreadPool(10); Future[] checkpointerResults = new Future[NUM_CHECKPOINTERS]; @@ -123,13 +113,8 @@ public void testDataSourceTask() throws Exception { checkpointerResults[i] = executor.submit(new Checkpointer(NUM_CHECKPOINTS, CHECKPOINT_INTERVAL, sourceTask)); } - - try { - sourceTask.invoke(); - } catch (Exception e) { - System.err.println(StringUtils.stringifyException(e)); - Assert.fail("Invoke method caused exception."); - } + testHarness.invoke(); + testHarness.waitForTaskCompletion(); // Get the result from the checkpointers, if these threw an exception it // will be rethrown here @@ -142,11 +127,11 @@ public void testDataSourceTask() throws Exception { } } - Assert.assertEquals(NUM_ELEMENTS, outList.size()); + List> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); + Assert.assertEquals(NUM_ELEMENTS, resultElements.size()); } - private static class MockSource extends RichSourceFunction> implements StateCheckpointer { - + private static class MockSource implements SourceFunction>, Checkpointed { private static final long serialVersionUID = 1; private int maxElements; @@ -157,7 +142,6 @@ private static class MockSource extends RichSourceFunction private volatile long lastCheckpointId = -1; private Semaphore semaphore; - private OperatorState state; private volatile boolean isRunning = true; @@ -166,7 +150,7 @@ public MockSource(int maxElements, int checkpointDelay, int readDelay) { this.checkpointDelay = checkpointDelay; this.readDelay = readDelay; this.count = 0; - this.semaphore = new Semaphore(1); + semaphore = new Semaphore(1); } @Override @@ -191,33 +175,32 @@ public void run(SourceContext> ctx) { public void cancel() { isRunning = false; } - - @Override - public void open(Configuration conf) throws IOException{ - state = getRuntimeContext().getOperatorState("state", 1, false, this); - } - @Override - public Integer snapshotState(Integer state, long checkpointId, long checkpointTimestamp) { + public Serializable snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { if (!semaphore.tryAcquire()) { Assert.fail("Concurrent invocation of snapshotState."); - } else { - int startCount = count; - - if (startCount != count) { - semaphore.release(); - // This means that next() was invoked while the snapshot was ongoing - Assert.fail("Count is different at start end end of snapshot."); - } + } + int startCount = count; + lastCheckpointId = checkpointId; + + long sum = 0; + for (int i = 0; i < checkpointDelay; i++) { + sum += new Random().nextLong(); + } + + if (startCount != count) { semaphore.release(); + // This means that next() was invoked while the snapshot was ongoing + Assert.fail("Count is different at start end end of snapshot."); } - return 0; + semaphore.release(); + return sum; } @Override - public Integer restoreState(Integer stateSnapshot) { - return stateSnapshot; + public void restoreState(Serializable state) { + } } @@ -247,5 +230,45 @@ public Boolean call() throws Exception { return true; } } + + public static class OpenCloseTestSource extends RichSourceFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public void run(SourceContext ctx) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + for (int i = 0; i < 10; i++) { + ctx.collect("Hello" + i); + } + } + + @Override + public void cancel() { + + } + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index 2092d839d8ce6..df0c9ee7f3bce 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -25,6 +25,7 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.event.task.AbstractEvent; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -36,7 +37,6 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.io.network.partition.consumer.IteratorWrappingTestSingleInputGate; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; @@ -45,8 +45,6 @@ import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.runtime.state.StateHandle; -import org.apache.flink.types.Record; -import org.apache.flink.util.MutableObjectIterator; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -54,6 +52,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Queue; import java.util.concurrent.Future; import static org.junit.Assert.fail; @@ -87,9 +86,9 @@ public class StreamMockEnvironment implements Environment { private final int bufferSize; - public StreamMockEnvironment(long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) { - this.jobConfiguration = new Configuration(); - this.taskConfiguration = new Configuration(); + public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) { + this.jobConfiguration = jobConfig; + this.taskConfiguration = taskConfig; this.inputs = new LinkedList(); this.outputs = new LinkedList(); @@ -101,20 +100,11 @@ public StreamMockEnvironment(long memorySize, MockInputSplitProvider inputSplitP this.accumulatorRegistry = new AccumulatorRegistry(jobID, getExecutionId()); } - public IteratorWrappingTestSingleInputGate addInput(MutableObjectIterator inputIterator) { - try { - final IteratorWrappingTestSingleInputGate reader = new IteratorWrappingTestSingleInputGate(bufferSize, Record.class, inputIterator); - - inputs.add(reader.getInputGate()); - - return reader; - } - catch (Throwable t) { - throw new RuntimeException("Error setting up mock readers: " + t.getMessage(), t); - } + public void addInputGate(InputGate gate) { + inputs.add(gate); } - public void addOutput(final List outputList, final TypeSerializer serializer) { + public void addOutput(final Queue outputList, final TypeSerializer serializer) { try { // The record-oriented writers wrap the buffer writer. We mock it // to collect the returned buffers and deserialize the content to @@ -161,6 +151,29 @@ public Void answer(InvocationOnMock invocationOnMock) throws Throwable { } }).when(mockWriter).writeBuffer(any(Buffer.class), anyInt()); + // Add events to the output list + doAnswer(new Answer() { + + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + AbstractEvent event = (AbstractEvent) invocationOnMock.getArguments()[0]; + + outputList.add(event); + return null; + } + }).when(mockWriter).writeEvent(any(AbstractEvent.class), anyInt()); + + doAnswer(new Answer() { + + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + AbstractEvent event = (AbstractEvent) invocationOnMock.getArguments()[0]; + + outputList.add(event); + return null; + } + }).when(mockWriter).writeEventToAllChannels(any(AbstractEvent.class)); + outputs.add(mockWriter); } catch (Throwable t) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestBase.java deleted file mode 100644 index f1a36c8593ec8..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestBase.java +++ /dev/null @@ -1,109 +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.runtime.tasks; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.io.network.partition.consumer.IteratorWrappingTestSingleInputGate; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.types.Record; -import org.apache.flink.util.MutableObjectIterator; -import org.junit.After; -import org.junit.Assert; - -import java.util.List; - - -public abstract class StreamTaskTestBase { - - protected long memorySize = 0; - - protected StreamMockEnvironment mockEnv; - - public void initEnvironment(long memorySize, int bufferSize) { - this.memorySize = memorySize; - this.mockEnv = new StreamMockEnvironment(this.memorySize, new MockInputSplitProvider(), bufferSize); - } - - public IteratorWrappingTestSingleInputGate addInput(MutableObjectIterator input, int groupId) { - final IteratorWrappingTestSingleInputGate reader = addInput(input, groupId, true); - - return reader; - } - - public IteratorWrappingTestSingleInputGate addInput(MutableObjectIterator input, int groupId, boolean read) { - final IteratorWrappingTestSingleInputGate reader = this.mockEnv.addInput(input); - TaskConfig conf = new TaskConfig(this.mockEnv.getTaskConfiguration()); - conf.addInputToGroup(groupId); - conf.setInputSerializer(RecordSerializerFactory.get(), groupId); - - if (read) { - reader.read(); - } - - return reader; - } - - public void addOutput(List output, TypeSerializer serializer) { - this.mockEnv.addOutput(output, serializer); - TaskConfig conf = new TaskConfig(this.mockEnv.getTaskConfiguration()); - conf.addOutputShipStrategy(ShipStrategyType.FORWARD); - conf.setOutputSerializer(RecordSerializerFactory.get()); - } - - public Configuration getConfiguration() { - return this.mockEnv.getTaskConfiguration(); - } - - public StreamConfig getStreamConfig() { - return new StreamConfig(this.mockEnv.getTaskConfiguration()); - } - - public void registerTask(AbstractInvokable task) { - task.setEnvironment(this.mockEnv); - task.registerInputOutput(); - } - - public MemoryManager getMemoryManager() { - return this.mockEnv.getMemoryManager(); - } - - @After - public void shutdownIOManager() throws Exception { - this.mockEnv.getIOManager().shutdown(); - Assert.assertTrue("IO Manager has not properly shut down.", this.mockEnv.getIOManager().isProperlyShutDown()); - } - - @After - public void shutdownMemoryManager() throws Exception { - if (this.memorySize > 0) { - MemoryManager memMan = getMemoryManager(); - if (memMan != null) { - Assert.assertTrue("Memory Manager managed memory was not completely freed.", memMan.verifyEmpty()); - memMan.shutdown(); - } - } - } -} - diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java new file mode 100644 index 0000000000000..a4cc0d3fb7ec1 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -0,0 +1,308 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.runtime.tasks; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.event.task.AbstractEvent; +import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.streaming.api.collector.selector.BroadcastOutputSelectorWrapper; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.api.graph.StreamNode; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; +import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; +import org.apache.flink.util.InstantiationUtil; +import org.junit.Assert; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * Test harness for testing a {@link StreamTask}. + * + *

+ * This mock Invokable provides the task with a basic runtime context and allows pushing elements + * and watermarks into the task. {@link #getOutput()} can be used to get the emitted elements + * and events. You are free to modify the retrieved list. + * + *

+ * After setting up everything the Task can be invoked using {@link #invoke()}. This will start + * a new Thread to execute the Task. Use {@link #waitForTaskCompletion()} to wait for the Task + * thread to finish. + * + *

+ * When using this you need to add the following line to your test class to setup Powermock: + * {@code @PrepareForTest({ResultPartitionWriter.class})} + */ +public class StreamTaskTestHarness { + + private static final int DEFAULT_MEMORY_MANAGER_SIZE = 1024 * 1024; + + private static final int DEFAULT_NETWORK_BUFFER_SIZE = 1024; + + protected long memorySize = 0; + protected int bufferSize = 0; + + protected StreamMockEnvironment mockEnv; + protected ExecutionConfig executionConfig; + private Configuration jobConfig; + private Configuration taskConfig; + protected StreamConfig streamConfig; + + private AbstractInvokable task; + + private TypeInformation outputType; + private TypeSerializer outputSerializer; + private StreamRecordSerializer outputStreamRecordSerializer; + + private ConcurrentLinkedQueue outputList; + + protected Thread taskThread; + + // These don't get initialized, the one-input/two-input specific test harnesses + // must initialize these if they want to simulate input. We have them here so that all the + // input related methods only need to be implemented once, in generic form + protected int numInputGates; + protected int numInputChannelsPerGate; + protected StreamTestSingleInputGate[] inputGates; + + public StreamTaskTestHarness(AbstractInvokable task, TypeInformation outputType) { + this.task = task; + this.memorySize = DEFAULT_MEMORY_MANAGER_SIZE; + this.bufferSize = DEFAULT_NETWORK_BUFFER_SIZE; + + this.jobConfig = new Configuration(); + this.taskConfig = new Configuration(); + this.executionConfig = new ExecutionConfig(); + executionConfig.enableTimestamps(); + try { + InstantiationUtil.writeObjectToConfig(executionConfig, this.jobConfig, ExecutionConfig.CONFIG_KEY); + } catch (IOException e) { + throw new RuntimeException(e); + } + + streamConfig = new StreamConfig(taskConfig); + streamConfig.setChainStart(); + streamConfig.setBufferTimeout(0); + + this.outputType = outputType; + outputSerializer = outputType.createSerializer(executionConfig); + outputStreamRecordSerializer = new MultiplexingStreamRecordSerializer(outputSerializer); + } + + /** + * This must be overwritten for OneInputStreamTask or TwoInputStreamTask test harnesses. + */ + protected void initializeInputs() throws IOException, InterruptedException { + } + + @SuppressWarnings("unchecked") + private void initializeOutput() { + outputList = new ConcurrentLinkedQueue(); + + mockEnv.addOutput(outputList, outputStreamRecordSerializer); + + streamConfig.setOutputSelectorWrapper(new BroadcastOutputSelectorWrapper()); + streamConfig.setNumberOfOutputs(1); + + StreamOperator dummyOperator = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + List outEdgesInOrder = new LinkedList(); + StreamNode sourceVertexDummy = new StreamNode(null, 0, dummyOperator, "source dummy", new LinkedList>(), SourceStreamTask.class); + StreamNode targetVertexDummy = new StreamNode(null, 1, dummyOperator, "target dummy", new LinkedList>(), SourceStreamTask.class); + + outEdgesInOrder.add(new StreamEdge(sourceVertexDummy, targetVertexDummy, 0, new LinkedList(), new BroadcastPartitioner())); + streamConfig.setOutEdgesInOrder(outEdgesInOrder); + streamConfig.setNonChainedOutputs(outEdgesInOrder); + streamConfig.setTypeSerializerOut1(outputSerializer); + streamConfig.setVertexID(0); + + } + + /** + * Invoke the Task. This resets the output of any previous invocation. This will start a new + * Thread to execute the Task in. Use {@link #waitForTaskCompletion()} to wait for the + * Task thread to finish running. + */ + public void invoke() throws Exception { + mockEnv = new StreamMockEnvironment(jobConfig, taskConfig, memorySize, new MockInputSplitProvider(), bufferSize); + task.setEnvironment(mockEnv); + + initializeInputs(); + initializeOutput(); + + task.registerInputOutput(); + + taskThread = new Thread(new Runnable() { + @Override + public void run() { + + + + try { + task.invoke(); + shutdownIOManager(); + shutdownMemoryManager(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + }); + + taskThread.start(); + } + + public void waitForTaskCompletion() throws InterruptedException { + if (taskThread == null) { + throw new IllegalStateException("Task thread was not started."); + } + + taskThread.join(); + } + + /** + * Get all the output from the task. This contains StreamRecords and Events interleaved. Use + * {@link org.apache.flink.streaming.util.TestHarnessUtil#getRawElementsFromOutput(java.util.Queue)}} + * to extract only the StreamRecords. + */ + public Queue getOutput() { + return outputList; + } + + public StreamConfig getStreamConfig() { + return streamConfig; + } + + private void shutdownIOManager() throws Exception { + this.mockEnv.getIOManager().shutdown(); + Assert.assertTrue("IO Manager has not properly shut down.", this.mockEnv.getIOManager().isProperlyShutDown()); + } + + private void shutdownMemoryManager() throws Exception { + if (this.memorySize > 0) { + MemoryManager memMan = this.mockEnv.getMemoryManager(); + if (memMan != null) { + Assert.assertTrue("Memory Manager managed memory was not completely freed.", memMan.verifyEmpty()); + memMan.shutdown(); + } + } + } + + /** + * Sends the element to input gate 0 on channel 0. + */ + @SuppressWarnings("unchecked") + public void processElement(Object element) { + inputGates[0].sendElement(element, 0); + } + + /** + * Sends the element to the specified channel on the specified input gate. + */ + @SuppressWarnings("unchecked") + public void processElement(Object element, int inputGate, int channel) { + inputGates[inputGate].sendElement(element, channel); + } + + /** + * Sends the event to input gate 0 on channel 0. + */ + public void processEvent(AbstractEvent event) { + inputGates[0].sendEvent(event, 0); + } + + /** + * Sends the event to the specified channel on the specified input gate. + */ + public void processEvent(AbstractEvent event, int inputGate, int channel) { + inputGates[inputGate].sendEvent(event, channel); + } + + /** + * This only returns after all input queues are empty. + */ + public void waitForInputProcessing() { + + + // first wait for all input queues to be empty + try { + Thread.sleep(1); + } catch (InterruptedException e) { + } + while (true) { + boolean allEmpty = true; + for (int i = 0; i < numInputGates; i++) { + if (!inputGates[i].allQueuesEmpty()) { + allEmpty = false; + } + } + try { + Thread.sleep(10); + } catch (InterruptedException e) { + } + if (allEmpty) { + break; + } + } + + // then wait for the Task Thread to be in a blocked state + // Check whether the state is blocked, this should be the case if it cannot + // read more input, i.e. all currently available input has been processed. + while (true) { + Thread.State state = taskThread.getState(); + if (state == Thread.State.BLOCKED || state == Thread.State.TERMINATED || + state == Thread.State.WAITING || state == Thread.State.TIMED_WAITING) { + break; + } + + try { + Thread.sleep(1); + } catch (InterruptedException e) { + } + } + } + + /** + * Notifies all input channels on all input gates that no more input will arrive. This + * will usually make the Task exit from his internal loop. + */ + public void endInput() { + for (int i = 0; i < numInputGates; i++) { + inputGates[i].endInput(); + } + } +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java new file mode 100644 index 0000000000000..3b113ab07c7e7 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -0,0 +1,374 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.runtime.tasks; + + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.streaming.api.functions.co.CoMapFunction; +import org.apache.flink.streaming.api.functions.co.RichCoMapFunction; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.co.CoStreamMap; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * Tests for {@link org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask}. Theses tests + * implicitly also test the {@link org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor}. + * + *

+ * Note:
+ * We only use a {@link CoStreamMap} operator here. We also test the individual operators but Map is + * used as a representative to test TwoInputStreamTask, since TwoInputStreamTask is used for all + * TwoInputStreamOperators. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({ResultPartitionWriter.class}) +public class TwoInputStreamTaskTest { + + /** + * This test verifies that open() and close() are correctly called. This test also verifies + * that timestamps of emitted elements are correct. {@link CoStreamMap} assigns the input + * timestamp to emitted elements. + */ + @Test + @SuppressWarnings("unchecked") + public void testOpenCloseAndTimestamps() throws Exception { + final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); + final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness(coMapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + CoStreamMap coMapOperator = new CoStreamMap(new TestOpenCloseMapFunction()); + streamConfig.setStreamOperator(coMapOperator); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + + testHarness.invoke(); + + testHarness.processElement(new StreamRecord("Hello", initialTime + 1), 0, 0); + expectedOutput.add(new StreamRecord("Hello", initialTime + 1)); + + // wait until the input is processed to ensure ordering of the output + testHarness.waitForInputProcessing(); + + testHarness.processElement(new StreamRecord(1337, initialTime + 2), 1, 0); + + expectedOutput.add(new StreamRecord("1337", initialTime + 2)); + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseMapFunction.closeCalled); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + /** + * This test verifies that watermarks are correctly forwarded. This also checks whether + * watermarks are forwarded only when we have received watermarks from all inputs. The + * forwarded watermark must be the minimum of the watermarks of all inputs. + */ + @Test + @SuppressWarnings("unchecked") + public void testWatermarkForwarding() throws Exception { + final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); + final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); + streamConfig.setStreamOperator(coMapOperator); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + long initialTime = 0L; + + testHarness.invoke(); + + testHarness.processElement(new Watermark(initialTime), 0, 0); + testHarness.processElement(new Watermark(initialTime), 0, 1); + + testHarness.processElement(new Watermark(initialTime), 1, 0); + + + // now the output should still be empty + testHarness.waitForInputProcessing(); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.processElement(new Watermark(initialTime), 1, 1); + + // now the watermark should have propagated, Map simply forward Watermarks + testHarness.waitForInputProcessing(); + expectedOutput.add(new Watermark(initialTime)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + // contrary to checkpoint barriers these elements are not blocked by watermarks + testHarness.processElement(new StreamRecord("Hello", initialTime), 0, 0); + testHarness.processElement(new StreamRecord(42, initialTime), 1, 1); + expectedOutput.add(new StreamRecord("Hello", initialTime)); + expectedOutput.add(new StreamRecord("42", initialTime)); + + testHarness.waitForInputProcessing(); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.processElement(new Watermark(initialTime + 4), 0, 0); + testHarness.processElement(new Watermark(initialTime + 3), 0, 1); + testHarness.processElement(new Watermark(initialTime + 3), 1, 0); + testHarness.processElement(new Watermark(initialTime + 2), 1, 1); + + // check whether we get the minimum of all the watermarks, this must also only occur in + // the output after the two StreamRecords + expectedOutput.add(new Watermark(initialTime + 2)); + testHarness.waitForInputProcessing(); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + + // advance watermark from one of the inputs, now we should get a now one since the + // minimum increases + testHarness.processElement(new Watermark(initialTime + 4), 1, 1); + testHarness.waitForInputProcessing(); + expectedOutput.add(new Watermark(initialTime + 3)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + // advance the other two inputs, now we should get a new one since the + // minimum increases again + testHarness.processElement(new Watermark(initialTime + 4), 0, 1); + testHarness.processElement(new Watermark(initialTime + 4), 1, 0); + testHarness.waitForInputProcessing(); + expectedOutput.add(new Watermark(initialTime + 4)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + List resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); + Assert.assertEquals(2, resultElements.size()); + } + + /** + * This test verifies that checkpoint barriers are correctly forwarded. + */ + @Test + @SuppressWarnings("unchecked") + public void testCheckpointBarriers() throws Exception { + final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); + final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); + streamConfig.setStreamOperator(coMapOperator); + + Queue expectedOutput = new ConcurrentLinkedQueue(); + long initialTime = 0L; + + testHarness.invoke(); + + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0); + + // This element should be buffered since we received a checkpoint barrier on + // this input + testHarness.processElement(new StreamRecord("Hello-0-0", initialTime), 0, 0); + + // This one should go through + testHarness.processElement(new StreamRecord("Ciao-0-0", initialTime), 0, 1); + expectedOutput.add(new StreamRecord("Ciao-0-0", initialTime)); + + // These elements should be forwarded, since we did not yet receive a checkpoint barrier + // on that input, only add to same input, otherwise we would not know the ordering + // of the output since the Task might read the inputs in any order + testHarness.processElement(new StreamRecord(11, initialTime), 1, 1); + testHarness.processElement(new StreamRecord(111, initialTime), 1, 1); + expectedOutput.add(new StreamRecord("11", initialTime)); + expectedOutput.add(new StreamRecord("111", initialTime)); + + testHarness.waitForInputProcessing(); + // we should not yet see the barrier, only the two elements from non-blocked input + TestHarnessUtil.assertOutputEquals("Output was not correct.", + testHarness.getOutput(), + expectedOutput); + + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1); + + testHarness.waitForInputProcessing(); + + // now we should see the barrier and after that the buffered elements + expectedOutput.add(new CheckpointBarrier(0, 0)); + expectedOutput.add(new StreamRecord("Hello-0-0", initialTime)); + TestHarnessUtil.assertOutputEquals("Output was not correct.", + testHarness.getOutput(), + expectedOutput); + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + List resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); + Assert.assertEquals(4, resultElements.size()); + } + + /** + * This test verifies that checkpoint barriers and barrier buffers work correctly with + * concurrent checkpoint barriers where one checkpoint is "overtaking" another checkpoint, i.e. + * some inputs receive barriers from an earlier checkpoint, thereby blocking, + * then all inputs receive barriers from a later checkpoint. + */ + @Test + @SuppressWarnings("unchecked") + public void testOvertakingCheckpointBarriers() throws Exception { + final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); + final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + StreamConfig streamConfig = testHarness.getStreamConfig(); + CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); + streamConfig.setStreamOperator(coMapOperator); + + Queue expectedOutput = new ConcurrentLinkedQueue(); + long initialTime = 0L; + + testHarness.invoke(); + + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0); + + // These elements should be buffered until we receive barriers from + // all inputs + testHarness.processElement(new StreamRecord("Hello-0-0", initialTime), 0, 0); + testHarness.processElement(new StreamRecord("Ciao-0-0", initialTime), 0, 0); + + // These elements should be forwarded, since we did not yet receive a checkpoint barrier + // on that input, only add to same input, otherwise we would not know the ordering + // of the output since the Task might read the inputs in any order + testHarness.processElement(new StreamRecord(42, initialTime), 1, 1); + testHarness.processElement(new StreamRecord(1337, initialTime), 1, 1); + expectedOutput.add(new StreamRecord("42", initialTime)); + expectedOutput.add(new StreamRecord("1337", initialTime)); + + testHarness.waitForInputProcessing(); + // we should not yet see the barrier, only the two elements from non-blocked input + TestHarnessUtil.assertOutputEquals("Output was not correct.", + expectedOutput, + testHarness.getOutput()); + + // Now give a later barrier to all inputs, this should unblock the first channel, + // thereby allowing the two blocked elements through + testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 0); + testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 1); + testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 0); + testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 1); + + expectedOutput.add(new StreamRecord("Hello-0-0", initialTime)); + expectedOutput.add(new StreamRecord("Ciao-0-0", initialTime)); + expectedOutput.add(new CheckpointBarrier(1, 1)); + + testHarness.waitForInputProcessing(); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", + expectedOutput, + testHarness.getOutput()); + + + // Then give the earlier barrier, these should be ignored + testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0); + testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1); + + testHarness.waitForInputProcessing(); + + + testHarness.endInput(); + + testHarness.waitForTaskCompletion(); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", + expectedOutput, + testHarness.getOutput()); + } + + // This must only be used in one test, otherwise the static fields will be changed + // by several tests concurrently + private static class TestOpenCloseMapFunction extends RichCoMapFunction { + private static final long serialVersionUID = 1L; + + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + Assert.fail("Close called before open."); + } + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + Assert.fail("Open was not called before close."); + } + closeCalled = true; + } + + @Override + public String map1(String value) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return value; + } + + @Override + public String map2(Integer value) throws Exception { + if (!openCalled) { + Assert.fail("Open was not called before run."); + } + return value.toString(); + } + } + + private static class IdentityMap implements CoMapFunction { + private static final long serialVersionUID = 1L; + + @Override + public String map1(String value) throws Exception { + return value; + } + + @Override + public String map2(Integer value) throws Exception { + + return value.toString(); + } + } +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java new file mode 100644 index 0000000000000..f37eb6632bb13 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java @@ -0,0 +1,170 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.runtime.tasks; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; +import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.api.graph.StreamNode; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; + + +/** + * Test harness for testing a {@link TwoInputStreamTask}. + * + *

+ * This mock Invokable provides the task with a basic runtime context and allows pushing elements + * and watermarks into the task. {@link #getOutput()} can be used to get the emitted elements + * and events. You are free to modify the retrieved list. + * + *

+ * After setting up everything the Task can be invoked using {@link #invoke()}. This will start + * a new Thread to execute the Task. Use {@link #waitForTaskCompletion()} to wait for the Task + * thread to finish. Use {@link #processElement} + * to send elements to the task. Use + * {@link #processEvent(org.apache.flink.runtime.event.task.AbstractEvent)} to send events to the task. + * Before waiting for the task to finish you must call {@link #endInput()} to signal to the task + * that data entry is finished. + * + *

+ * When Elements or Events are offered to the Task they are put into a queue. The input gates + * of the Task read from this queue. Use {@link #waitForInputProcessing()} to wait until all + * queues are empty. This must be used after entering some elements before checking the + * desired output. + * + *

+ * When using this you need to add the following line to your test class to setup Powermock: + * {@code @PrepareForTest({ResultPartitionWriter.class})} + */ +public class TwoInputStreamTaskTestHarness extends StreamTaskTestHarness { + + private TypeInformation inputType1; + private TypeSerializer inputSerializer1; + + private TypeInformation inputType2; + private TypeSerializer inputSerializer2; + + private int[] inputGateAssignment; + + /** + * Creates a test harness with the specified number of input gates and specified number + * of channels per input gate. Parameter inputGateAssignment specifies for each gate whether + * it should be assigned to the first (1), or second (2) input of the task. + */ + public TwoInputStreamTaskTestHarness(TwoInputStreamTask task, + int numInputGates, + int numInputChannelsPerGate, + int[] inputGateAssignment, + TypeInformation inputType1, + TypeInformation inputType2, + TypeInformation outputType) { + super(task, outputType); + + this.inputType1 = inputType1; + inputSerializer1 = inputType1.createSerializer(executionConfig); + + this.inputType2 = inputType2; + inputSerializer2 = inputType2.createSerializer(executionConfig); + + this.numInputGates = numInputGates; + this.numInputChannelsPerGate = numInputChannelsPerGate; + this.inputGateAssignment = inputGateAssignment; + } + + /** + * Creates a test harness with one input gate (that has one input channel) per input. The first + * input gate is assigned to the first task input, the second input gate is assigned to the + * second task input. + */ + public TwoInputStreamTaskTestHarness(TwoInputStreamTask task, + TypeInformation inputType1, + TypeInformation inputType2, + TypeInformation outputType) { + this(task, 2, 1, new int[] {1, 2}, inputType1, inputType2, outputType); + } + + @Override + protected void initializeInputs() throws IOException, InterruptedException { + + inputGates = new StreamTestSingleInputGate[numInputGates]; + List inPhysicalEdges = new LinkedList(); + + StreamOperator dummyOperator = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + StreamNode sourceVertexDummy = new StreamNode(null, 0, dummyOperator, "source dummy", new LinkedList>(), SourceStreamTask.class); + StreamNode targetVertexDummy = new StreamNode(null, 1, dummyOperator, "target dummy", new LinkedList>(), SourceStreamTask.class); + + for (int i = 0; i < numInputGates; i++) { + + switch (inputGateAssignment[i]) { + case 1: { + inputGates[i] = new StreamTestSingleInputGate( + numInputChannelsPerGate, + bufferSize, + inputSerializer1); + + + StreamEdge streamEdge = new StreamEdge(sourceVertexDummy, + targetVertexDummy, + 1, + new LinkedList(), + new BroadcastPartitioner()); + + inPhysicalEdges.add(streamEdge); + break; + } + case 2: { + inputGates[i] = new StreamTestSingleInputGate( + numInputChannelsPerGate, + bufferSize, + inputSerializer2); + + StreamEdge streamEdge = new StreamEdge(sourceVertexDummy, + targetVertexDummy, + 2, + new LinkedList(), + new BroadcastPartitioner()); + + inPhysicalEdges.add(streamEdge); + break; + } + default: + throw new IllegalStateException("Wrong input gate assignment."); + } + + this.mockEnv.addInputGate(inputGates[i].getInputGate()); + } + + streamConfig.setInPhysicalEdges(inPhysicalEdges); + streamConfig.setNumberOfInputs(numInputGates); + streamConfig.setTypeSerializerIn1(inputSerializer1); + streamConfig.setTypeSerializerIn2(inputSerializer2); + } + +} + diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java new file mode 100644 index 0000000000000..d3fde9e25bb21 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java @@ -0,0 +1,416 @@ +/** + * 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.timestamp; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.co.CoMapFunction; +import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.fail; + +/** + * Tests for timestamps, watermarks, and event-time sources. + */ +@SuppressWarnings("serial") +public class TimestampITCase { + + private static final int NUM_TASK_MANAGERS = 2; + private static final int NUM_TASK_SLOTS = 3; + private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + + private static ForkableFlinkMiniCluster cluster; + + @BeforeClass + public static void startCluster() { + try { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); + config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + + cluster = new ForkableFlinkMiniCluster(config, false); + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to start test cluster: " + e.getMessage()); + } + } + + @AfterClass + public static void shutdownCluster() { + try { + cluster.shutdown(); + cluster = null; + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to stop test cluster: " + e.getMessage()); + } + } + + /** + * These check whether custom timestamp emission works at sources and also whether timestamps + * arrive at operators throughout a topology. + * + *

+ * This only uses map to test the workings of watermarks in a complete, running topology. All + * tasks and stream operators have dedicated tests that test the watermark propagation + * behaviour. + */ + @Test + public void testWatermarkPropagation() throws Exception { + final int NUM_WATERMARKS = 10; + + long initialTime = 0L; + + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.getConfig().disableSysoutLogging(); + env.getConfig().enableTimestamps(); + + + DataStream source1 = env.addSource(new MyTimestampSource(initialTime, NUM_WATERMARKS)); + DataStream source2 = env.addSource(new MyTimestampSource(initialTime, NUM_WATERMARKS)); + + source1 + .map(new IdentityMap()) + .connect(source2).map(new IdentityCoMap()) + .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator()); + + env.execute(); + + // verify that all the watermarks arrived at the final custom operator + for (int i = 0; i < PARALLELISM; i++) { + for (int j = 0; j < NUM_WATERMARKS; j++) { + if (!CustomOperator.finalWatermarks[i].get(j).equals(new Watermark(initialTime + j))) { + Assert.fail("Wrong watermark."); + } + } + } + } + + + + /** + * These check whether timestamps are properly assigned at the sources and handled in + * network transmission and between chained operators when timestamps are enabled. + */ + @Test + public void testTimestampHandling() throws Exception { + final int NUM_ELEMENTS = 10; + + + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.getConfig().disableSysoutLogging(); + env.getConfig().enableTimestamps(); + + + DataStream source1 = env.addSource(new MyTimestampSource(0L, NUM_ELEMENTS)); + DataStream source2 = env.addSource(new MyTimestampSource(0L, NUM_ELEMENTS)); + + source1 + .map(new IdentityMap()) + .connect(source2).map(new IdentityCoMap()) + .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator()); + + env.execute(); + } + + /** + * These check whether timestamps are properly ignored when they are disabled. + */ + @Test + public void testDisabledTimestamps() throws Exception { + final int NUM_ELEMENTS = 10; + + + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.getConfig().disableSysoutLogging(); + Assert.assertEquals("Timestamps are not disabled by default.", false, env.getConfig().areTimestampsEnabled()); + env.getConfig().disableTimestamps(); + + + DataStream source1 = env.addSource(new MyNonWatermarkingSource(NUM_ELEMENTS)); + DataStream source2 = env.addSource(new MyNonWatermarkingSource(NUM_ELEMENTS)); + + source1 + .map(new IdentityMap()) + .connect(source2).map(new IdentityCoMap()) + .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new DisabledTimestampCheckingOperator()); + + env.execute(); + } + + /** + * This tests whether the program throws an exception when an event-time source tries + * to emit without timestamp. + */ + @Test(expected = ProgramInvocationException.class) + public void testEventTimeSourceEmitWithoutTimestamp() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.getConfig().disableSysoutLogging(); + + DataStream source1 = env.addSource(new MyErroneousTimestampSource()); + + source1 + .map(new IdentityMap()); + + env.execute(); + } + + /** + * This tests whether the program throws an exception when a regular source tries + * to emit with timestamp. + */ + @Test(expected = ProgramInvocationException.class) + public void testSourceEmitWithTimestamp() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.getConfig().disableSysoutLogging(); + + DataStream source1 = env.addSource(new MyErroneousSource()); + + source1 + .map(new IdentityMap()); + + env.execute(); + } + + /** + * This tests whether the program throws an exception when a regular source tries + * to emit a watermark. + */ + @Test(expected = ProgramInvocationException.class) + public void testSourceEmitWatermark() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.getConfig().disableSysoutLogging(); + + DataStream source1 = env.addSource(new MyErroneousWatermarkSource()); + + source1 + .map(new IdentityMap()); + + env.execute(); + } + + public static class CustomOperator extends AbstractStreamOperator implements OneInputStreamOperator { + + List watermarks; + public static List[] finalWatermarks = new List[PARALLELISM]; + private long oldTimestamp; + + @Override + public void processElement(StreamRecord element) throws Exception { + if (element.getTimestamp() != element.getValue()) { + Assert.fail("Timestamps are not properly handled."); + } + oldTimestamp = element.getTimestamp(); + output.collect(element); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + watermarks.add(mark); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + watermarks = new ArrayList(); + } + + @Override + public void close() throws Exception { + super.close(); + finalWatermarks[getRuntimeContext().getIndexOfThisSubtask()] = watermarks; + } + } + + public static class TimestampCheckingOperator extends AbstractStreamOperator implements OneInputStreamOperator { + + @Override + public void processElement(StreamRecord element) throws Exception { + if (element.getTimestamp() != element.getValue()) { + Assert.fail("Timestamps are not properly handled."); + } + output.collect(element); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + } + } + + public static class DisabledTimestampCheckingOperator extends AbstractStreamOperator implements OneInputStreamOperator { + + @Override + public void processElement(StreamRecord element) throws Exception { + if (element.getTimestamp() != 0) { + Assert.fail("Timestamps are not properly handled."); + } + output.collect(element); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + } + } + + public static class IdentityCoMap implements CoMapFunction { + @Override + public Integer map1(Integer value) throws Exception { + return value; + } + + @Override + public Integer map2(Integer value) throws Exception { + return value; + } + } + + public static class IdentityMap implements MapFunction { + @Override + public Integer map(Integer value) throws Exception { + return value; + } + } + + public static class MyTimestampSource implements EventTimeSourceFunction { + + long initialTime; + int numWatermarks; + + public MyTimestampSource(long initialTime, int numWatermarks) { + this.initialTime = initialTime; + this.numWatermarks = numWatermarks; + } + + @Override + public void run(SourceContext ctx) throws Exception { + for (int i = 0; i < numWatermarks; i++) { + ctx.collectWithTimestamp(i, initialTime + i); + ctx.emitWatermark(new Watermark(initialTime + i)); + } + } + + @Override + public void cancel() { + + } + } + + public static class MyNonWatermarkingSource implements SourceFunction { + + int numWatermarks; + + public MyNonWatermarkingSource(int numWatermarks) { + this.numWatermarks = numWatermarks; + } + + @Override + public void run(SourceContext ctx) throws Exception { + for (int i = 0; i < numWatermarks; i++) { + ctx.collect(i); + } + } + + @Override + public void cancel() { + + } + } + + // This is a event-time source. This should only emit elements with timestamps. The test should + // therefore throw an exception + public static class MyErroneousTimestampSource implements EventTimeSourceFunction { + + @Override + public void run(SourceContext ctx) throws Exception { + for (int i = 0; i < 10; i++) { + ctx.collect(i); + } + } + + @Override + public void cancel() { + + } + } + + // This is a normal source. This should only emit elements without timestamps. The test should + // therefore throw an exception + public static class MyErroneousSource implements SourceFunction { + + @Override + public void run(SourceContext ctx) throws Exception { + for (int i = 0; i < 10; i++) { + ctx.collectWithTimestamp(i, 0L); + } + } + + @Override + public void cancel() { + + } + } + + // This is a normal source. This should only emit elements without timestamps. This also + // must not emit watermarks. The test should therefore throw an exception + public static class MyErroneousWatermarkSource implements SourceFunction { + + @Override + public void run(SourceContext ctx) throws Exception { + for (int i = 0; i < 10; i++) { + ctx.collect(i); + ctx.emitWatermark(new Watermark(0L)); + } + } + + @Override + public void cancel() { + + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCoContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCoContext.java deleted file mode 100644 index 0467b5f26ee10..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCoContext.java +++ /dev/null @@ -1,216 +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; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; -import org.apache.flink.streaming.runtime.io.CoReaderIterator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; -import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; - -public class MockCoContext { - // private Collection input1; - // private Collection input2; - private Iterator inputIterator1; - private Iterator inputIterator2; - private List outputs; - - private Output collector; - private StreamRecordSerializer inDeserializer1; - private CoReaderIterator, StreamRecord> mockIterator; - private StreamRecordSerializer inDeserializer2; - - public MockCoContext(Collection input1, Collection input2) { - - if (input1.isEmpty() || input2.isEmpty()) { - throw new RuntimeException("Inputs must not be empty"); - } - - this.inputIterator1 = input1.iterator(); - this.inputIterator2 = input2.iterator(); - - TypeInformation inTypeInfo1 = TypeExtractor.getForObject(input1.iterator().next()); - inDeserializer1 = new StreamRecordSerializer(inTypeInfo1, new ExecutionConfig()); - TypeInformation inTypeInfo2 = TypeExtractor.getForObject(input2.iterator().next()); - inDeserializer2 = new StreamRecordSerializer(inTypeInfo2, new ExecutionConfig()); - - mockIterator = new MockCoReaderIterator(inDeserializer1, inDeserializer2); - - outputs = new ArrayList(); - collector = new MockOutput(outputs); - } - - private int currentInput = 1; - private StreamRecord reuse1; - private StreamRecord reuse2; - - private class MockCoReaderIterator extends - CoReaderIterator, StreamRecord> { - - public MockCoReaderIterator(TypeSerializer> serializer1, - TypeSerializer> serializer2) { - super(null, serializer1, serializer2); - reuse1 = inDeserializer1.createInstance(); - reuse2 = inDeserializer2.createInstance(); - } - - @Override - public int next(StreamRecord target1, StreamRecord target2) throws IOException { - this.delegate1.setInstance(target1); - this.delegate2.setInstance(target2); - - int inputNumber = nextRecord(); - target1.setObject(reuse1.getObject()); - target2.setObject(reuse2.getObject()); - - return inputNumber; - } - } - - private Integer nextRecord() { - if (inputIterator1.hasNext() && inputIterator2.hasNext()) { - switch (currentInput) { - case 1: - return next1(); - case 2: - return next2(); - default: - return 0; - } - } - - if (inputIterator1.hasNext()) { - return next1(); - } - - if (inputIterator2.hasNext()) { - return next2(); - } - - return 0; - } - - private int next1() { - reuse1 = inDeserializer1.createInstance(); - reuse1.setObject(inputIterator1.next()); - currentInput = 2; - return 1; - } - - private int next2() { - reuse2 = inDeserializer2.createInstance(); - reuse2.setObject(inputIterator2.next()); - currentInput = 1; - return 2; - } - - public List getOutputs() { - return outputs; - } - - public Output getCollector() { - return collector; - } - - public StreamRecordSerializer getInDeserializer1() { - return inDeserializer1; - } - - public StreamRecordSerializer getInDeserializer2() { - return inDeserializer2; - } - - public CoReaderIterator, StreamRecord> getIterator() { - return mockIterator; - } - - public static List createAndExecute(TwoInputStreamOperator operator, - List input1, List input2) { - MockCoContext mockContext = new MockCoContext(input1, input2); - StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext("CoMockTask", - new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), null, - new ExecutionConfig(), null, null, new HashMap>()); - - operator.setup(mockContext.collector, runtimeContext); - - try { - operator.open(null); - - StreamRecordSerializer inputDeserializer1 = mockContext.getInDeserializer1(); - StreamRecordSerializer inputDeserializer2 = mockContext.getInDeserializer2(); - CoReaderIterator, StreamRecord> coIter = mockContext.mockIterator; - - boolean isRunning = true; - - int next; - StreamRecord reuse1 = inputDeserializer1.createInstance(); - StreamRecord reuse2 = inputDeserializer2.createInstance(); - - while (isRunning) { - try { - next = coIter.next(reuse1, reuse2); - } catch (IOException e) { - if (isRunning) { - throw new RuntimeException("Could not read next record.", e); - } else { - // Task already cancelled do nothing - next = 0; - } - } catch (IllegalStateException e) { - if (isRunning) { - throw new RuntimeException("Could not read next record.", e); - } else { - // Task already cancelled do nothing - next = 0; - } - } - - if (next == 0) { - break; - } else if (next == 1) { - operator.processElement1(reuse1.getObject()); - reuse1 = inputDeserializer1.createInstance(); - } else { - operator.processElement2(reuse2.getObject()); - reuse2 = inputDeserializer2.createInstance(); - } - } - - operator.close(); - } catch (Exception e) { - throw new RuntimeException("Cannot invoke operator.", e); - } - - return mockContext.getOutputs(); - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java index 0d09c14e5e458..45ae88f5a0a5b 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java @@ -17,34 +17,25 @@ package org.apache.flink.streaming.util; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.io.IndexedReaderIterator; +import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; -import org.apache.flink.util.Collector; -import org.apache.flink.util.MutableObjectIterator; public class MockContext { private Collection inputs; private List outputs; private MockOutput output; - private StreamRecordSerializer inDeserializer; - private IndexedReaderIterator> iterator; public MockContext(Collection inputs) { this.inputs = inputs; @@ -52,58 +43,19 @@ public MockContext(Collection inputs) { throw new RuntimeException("Inputs must not be empty"); } - TypeInformation inTypeInfo = TypeExtractor.getForObject(inputs.iterator().next()); - inDeserializer = new StreamRecordSerializer(inTypeInfo, new ExecutionConfig()); - - iterator = new IndexedInputIterator(); outputs = new ArrayList(); output = new MockOutput(outputs); } - private class IndexedInputIterator extends IndexedReaderIterator> { - Iterator listIterator; - - public IndexedInputIterator() { - super(null, null); - listIterator = inputs.iterator(); - } - - @Override - public StreamRecord next(StreamRecord reuse) throws IOException { - if (listIterator.hasNext()) { - reuse.setObject(listIterator.next()); - } else { - reuse = null; - } - return reuse; - } - - @Override - public StreamRecord next() throws IOException { - if (listIterator.hasNext()) { - StreamRecord result = inDeserializer.createInstance(); - result.setObject(listIterator.next()); - return result; - } else { - return null; - } - } - } - public List getOutputs() { return outputs; } - public Collector getOutput() { + public Output> getOutput() { return output; } - public MutableObjectIterator> getIterator() { - return iterator; - } - - public static List createAndExecute(OneInputStreamOperator operator, - List inputs) { + public static List createAndExecute(OneInputStreamOperator operator, List inputs) { MockContext mockContext = new MockContext(inputs); StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext("MockTask", new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), null, @@ -114,8 +66,8 @@ public static List createAndExecute(OneInputStreamOperator nextRecord; - while ((nextRecord = mockContext.getIterator().next()) != null) { - operator.processElement(nextRecord.getObject()); + for (IN in: inputs) { + operator.processElement(new StreamRecord(in)); } operator.close(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java index 6799d870024e1..5371ba008724c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java @@ -22,9 +22,10 @@ import org.apache.commons.lang3.SerializationUtils; import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.util.Collector; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -public class MockOutput implements Output { +public class MockOutput implements Output> { private Collection outputs; public MockOutput(Collection outputs) { @@ -32,12 +33,17 @@ public MockOutput(Collection outputs) { } @Override - public void collect(T record) { + public void collect(StreamRecord record) { T copied = SerializationUtils.deserialize(SerializationUtils - .serialize((Serializable) record)); + .serialize((Serializable) record.getValue())); outputs.add(copied); } + @Override + public void emitWatermark(Watermark mark) { + throw new RuntimeException("THIS MUST BE IMPLEMENTED"); + } + @Override public void close() { } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java deleted file mode 100644 index 1731e7c00905b..0000000000000 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java +++ /dev/null @@ -1,40 +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; - -import static org.mockito.Mockito.doCallRealMethod; -import static org.mockito.Mockito.mock; - -import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.api.streamtask.MockRecordWriter; -import org.mockito.Mockito; - -public class MockRecordWriterFactory { - - @SuppressWarnings("unchecked") - public static MockRecordWriter create() { - MockRecordWriter recWriter = mock(MockRecordWriter.class); - - Mockito.when(recWriter.initList()).thenCallRealMethod(); - doCallRealMethod().when(recWriter).emit(Mockito.any(SerializationDelegate.class)); - - recWriter.initList(); - - return recWriter; - } -} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java new file mode 100644 index 0000000000000..133f1434a0429 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -0,0 +1,146 @@ +/** + * 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; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.LocalStateHandle; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; + +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * A test harness for testing a {@link OneInputStreamOperator}. + * + *

+ * This mock task provides the operator with a basic runtime context and allows pushing elements + * and watermarks into the operator. {@link java.util.Deque}s containing the emitted elements + * and watermarks can be retrieved. You are free to modify these. + */ +public class OneInputStreamOperatorTestHarness { + + OneInputStreamOperator operator; + + ConcurrentLinkedQueue outputList; + + ExecutionConfig executionConfig; + + public OneInputStreamOperatorTestHarness(OneInputStreamOperator operator) { + this.operator = operator; + + outputList = new ConcurrentLinkedQueue(); + + executionConfig = new ExecutionConfig(); + + StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext( + "MockTwoInputTask", + new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), + getClass().getClassLoader(), + executionConfig, + null, + new LocalStateHandle.LocalStateHandleProvider(), + new HashMap>()); + + operator.setup(new MockOutput(), runtimeContext); + } + + /** + * Get all the output from the task. This contains StreamRecords and Events interleaved. Use + * {@link org.apache.flink.streaming.util.TestHarnessUtil#getStreamRecordsFromOutput(java.util.List)} + * to extract only the StreamRecords. + */ + public Queue getOutput() { + return outputList; + } + + /** + * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(org.apache.flink.configuration.Configuration)} + * with an empty {@link org.apache.flink.configuration.Configuration}. + */ + public void open() throws Exception { + operator.open(new Configuration()); + } + + /** + * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(org.apache.flink.configuration.Configuration)} + * with the given {@link org.apache.flink.configuration.Configuration}. + */ + public void open(Configuration config) throws Exception { + operator.open(config); + } + + /** + * Calls close on the operator. + */ + public void close() throws Exception { + operator.close(); + } + + public void processElement(StreamRecord element) throws Exception { + operator.processElement(element); + } + + public void processElements(Collection> elements) throws Exception { + for (StreamRecord element: elements) { + operator.processElement(element); + } + } + + public void processWatermark(Watermark mark) throws Exception { + operator.processWatermark(mark); + } + + private class MockOutput implements Output> { + + private TypeSerializer outputSerializer; + + @Override + @SuppressWarnings("unchecked") + public void emitWatermark(Watermark mark) { + outputList.add(mark); + } + + @Override + @SuppressWarnings("unchecked") + public void collect(StreamRecord element) { + if (outputSerializer == null) { + outputSerializer = TypeExtractor.getForObject(element.getValue()).createSerializer(executionConfig); + } + outputList.add(new StreamRecord(outputSerializer.copy(element.getValue()), + element.getTimestamp())); + } + + @Override + public void close() { + // ignore + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java index 764fe5f8c5abb..2d7f6b5776b5e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java @@ -29,10 +29,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.state.LocalStateHandle.LocalStateHandleProvider; +import org.apache.flink.runtime.state.LocalStateHandle; +import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; -import org.apache.flink.util.Collector; public class SourceFunctionUtil { @@ -40,25 +43,20 @@ public static List runSourceFunction(SourceFunction sourceFunction) th List outputs = new ArrayList(); if (sourceFunction instanceof RichFunction) { RuntimeContext runtimeContext = new StreamingRuntimeContext("MockTask", new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), null, - new ExecutionConfig(), null, new LocalStateHandleProvider(), new HashMap>()); + new ExecutionConfig(), null, new LocalStateHandle.LocalStateHandleProvider(), new HashMap>()); ((RichFunction) sourceFunction).setRuntimeContext(runtimeContext); ((RichFunction) sourceFunction).open(new Configuration()); } try { - final Collector collector = new MockOutput(outputs); - final Object lockObject = new Object(); - SourceFunction.SourceContext ctx = new SourceFunction.SourceContext() { - @Override - public void collect(T element) { - collector.collect(element); - } - - @Override - public Object getCheckpointLock() { - return lockObject; - } - }; + final Output> collector = new MockOutput(outputs); + final Object lockingObject = new Object(); + SourceFunction.SourceContext ctx; + if (sourceFunction instanceof EventTimeSourceFunction) { + ctx = new StreamSource.ManualWatermarkContext(lockingObject, collector); + } else { + ctx = new StreamSource.NonWatermarkContext(lockingObject, collector); + } sourceFunction.run(ctx); } catch (Exception e) { throw new RuntimeException("Cannot invoke source.", e); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java new file mode 100644 index 0000000000000..a0a6c8d62498a --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java @@ -0,0 +1,68 @@ +/** + * 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; + +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.junit.Assert; + +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +/** + * Utils for working with the various test harnesses. + */ +public class TestHarnessUtil { + /** + * Extracts the StreamRecords from the given output list. + */ + @SuppressWarnings("unchecked") + public static List> getStreamRecordsFromOutput(List output) { + List> resultElements = new LinkedList>(); + for (Object e: output) { + if (e instanceof StreamRecord) { + resultElements.add((StreamRecord) e); + } + } + return resultElements; + } + + /** + * Extracts the raw elements from the given output list. + */ + @SuppressWarnings("unchecked") + public static List getRawElementsFromOutput(Queue output) { + List resultElements = new LinkedList(); + for (Object e: output) { + if (e instanceof StreamRecord) { + resultElements.add((OUT) ((StreamRecord) e).getValue()); + } + } + return resultElements; + } + + /** + * Compare the two queues containing operator/task output by converting them to an array first. + */ + public static void assertOutputEquals(String message, Queue expected, Queue actual) { + Assert.assertArrayEquals(message, + expected.toArray(), + actual.toArray()); + + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java new file mode 100644 index 0000000000000..ea753f83af029 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java @@ -0,0 +1,148 @@ +/** + * 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; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.LocalStateHandle; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * A test harness for testing a {@link TwoInputStreamOperator}. + * + *

+ * This mock task provides the operator with a basic runtime context and allows pushing elements + * and watermarks into the operator. {@link java.util.Deque}s containing the emitted elements + * and watermarks can be retrieved. you are free to modify these. + */ +public class TwoInputStreamOperatorTestHarness { + + TwoInputStreamOperator operator; + + ConcurrentLinkedQueue outputList; + + ExecutionConfig executionConfig; + + public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator operator) { + this.operator = operator; + + outputList = new ConcurrentLinkedQueue(); + + executionConfig = new ExecutionConfig(); + + StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext( + "MockTwoInputTask", + new MockEnvironment(3 * 1024 * 1024, new MockInputSplitProvider(), 1024), + getClass().getClassLoader(), + new ExecutionConfig(), + null, + new LocalStateHandle.LocalStateHandleProvider(), + new HashMap>()); + + operator.setup(new MockOutput(), runtimeContext); + } + + /** + * Get all the output from the task. This contains StreamRecords and Events interleaved. Use + * {@link org.apache.flink.streaming.util.TestHarnessUtil#getStreamRecordsFromOutput(java.util.List)} + * to extract only the StreamRecords. + */ + public Queue getOutput() { + return outputList; + } + + + /** + * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(Configuration)} + * with an empty {@link Configuration}. + */ + public void open() throws Exception { + operator.open(new Configuration()); + } + + /** + * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(Configuration)} + * with the given {@link Configuration}. + */ + public void open(Configuration config) throws Exception { + operator.open(config); + } + + /** + * Calls close on the operator. + */ + public void close() throws Exception { + operator.close(); + } + + public void processElement1(StreamRecord element) throws Exception { + operator.processElement1(element); + } + + public void processElement2(StreamRecord element) throws Exception { + operator.processElement2(element); + } + + public void processWatermark1(Watermark mark) throws Exception { + operator.processWatermark1(mark); + } + + public void processWatermark2(Watermark mark) throws Exception { + operator.processWatermark2(mark); + } + + private class MockOutput implements Output> { + + private TypeSerializer outputSerializer; + + @Override + @SuppressWarnings("unchecked") + public void emitWatermark(Watermark mark) { + outputList.add(mark); + } + + @Override + @SuppressWarnings("unchecked") + public void collect(StreamRecord element) { + if (outputSerializer == null) { + outputSerializer = TypeExtractor.getForObject(element.getValue()).createSerializer(executionConfig); + } + outputList.add(new StreamRecord(outputSerializer.copy(element.getValue()), + element.getTimestamp())); + } + + @Override + public void close() { + // ignore + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java new file mode 100644 index 0000000000000..619709260b0f0 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java @@ -0,0 +1,463 @@ +/* + * 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.checkpointing; + +import org.apache.flink.api.common.functions.RichFilterFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.functions.RichReduceFunction; +import org.apache.flink.api.common.state.OperatorState; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.util.Collector; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; + +import static org.junit.Assert.*; + +/** + * A simple test that runs a streaming topology with checkpointing enabled. This differs from + * {@link org.apache.flink.test.checkpointing.StreamCheckpointingITCase} in that it contains + * a TwoInput (or co-) Task. + * + *

+ * This checks whether checkpoint barriers correctly trigger TwoInputTasks and also whether + * this barriers are correctly forwarded. + * + *

+ * This uses a mixture of Operators with the {@link Checkpointed} interface and the new + * {@link org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext#getOperatorState} + * method. + * + *

+ * The test triggers a failure after a while and verifies that, after completion, the + * state reflects the "exactly once" semantics. + */ +@SuppressWarnings("serial") +public class CoStreamCheckpointingITCase { + + private static final int NUM_TASK_MANAGERS = 2; + private static final int NUM_TASK_SLOTS = 3; + private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + + private static ForkableFlinkMiniCluster cluster; + + @BeforeClass + public static void startCluster() { + try { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); + config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms"); + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); + + cluster = new ForkableFlinkMiniCluster(config, false); + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to start test cluster: " + e.getMessage()); + } + } + + @AfterClass + public static void shutdownCluster() { + try { + cluster.shutdown(); + cluster = null; + } + catch (Exception e) { + e.printStackTrace(); + fail("Failed to stop test cluster: " + e.getMessage()); + } + } + + + + /** + * Runs the following program: + * + *

+	 *     [ (source)->(filter)->(map) ] -> [ (co-map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
+	 * 
+ */ + @Test + public void runCheckpointedProgram() { + + final long NUM_STRINGS = 10000000L; + assertTrue("Broken test setup", NUM_STRINGS % 40 == 0); + + try { + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", cluster.getJobManagerRPCPort()); + env.setParallelism(PARALLELISM); + env.enableCheckpointing(500); + env.getConfig().disableSysoutLogging(); + + DataStream stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)); + + stream + // -------------- first vertex, chained to the source ---------------- + .filter(new StringRichFilterFunction()) + + // -------------- second vertex - the stateful one that also fails ---------------- + .connect(stream).flatMap(new LeftIdentityCoRichFlatMapFunction()) + + // -------------- third vertex - the stateful one that also fails ---------------- + .map(new StringPrefixCountRichMapFunction()) + .startNewChain() + .map(new StatefulCounterFunction()) + + // -------------- fourth vertex - reducer and the sink ---------------- + .groupBy("prefix") + .reduce(new OnceFailingReducer(NUM_STRINGS)) + .addSink(new RichSinkFunction() { + + private Map counts = new HashMap(); + + @Override + public void invoke(PrefixCount value) { + Character first = value.prefix.charAt(0); + Long previous = counts.get(first); + if (previous == null) { + counts.put(first, value.count); + } else { + counts.put(first, Math.max(previous, value.count)); + } + } + +// @Override +// public void close() { +// for (Long count : counts.values()) { +// assertEquals(NUM_STRINGS / 40, count.longValue()); +// } +// } + }); + + env.execute(); + + long filterSum = 0; + for (long l : StringRichFilterFunction.counts) { + filterSum += l; + } + + long coMapSum = 0; + for (long l : LeftIdentityCoRichFlatMapFunction.counts) { + coMapSum += l; + } + + long mapSum = 0; + for (long l : StringPrefixCountRichMapFunction.counts) { + mapSum += l; + } + + long countSum = 0; + for (long l : StatefulCounterFunction.counts) { + countSum += l; + } + + if (!StringPrefixCountRichMapFunction.restoreCalledAtLeastOnce) { + Assert.fail("Restore was never called on counting Map function."); + } + + if (!LeftIdentityCoRichFlatMapFunction.restoreCalledAtLeastOnce) { + Assert.fail("Restore was never called on counting CoMap function."); + } + + // verify that we counted exactly right + + assertEquals(NUM_STRINGS, filterSum); + assertEquals(NUM_STRINGS, coMapSum); + assertEquals(NUM_STRINGS, mapSum); + assertEquals(NUM_STRINGS, countSum); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + // Custom Functions + // -------------------------------------------------------------------------------------------- + + private static class StringGeneratingSourceFunction extends RichSourceFunction + implements ParallelSourceFunction { + + private final long numElements; + + private Random rnd; + private StringBuilder stringBuilder; + + private OperatorState index; + private int step; + + private volatile boolean isRunning; + + static final long[] counts = new long[PARALLELISM]; + @Override + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = index.value(); + } + + + StringGeneratingSourceFunction(long numElements) { + this.numElements = numElements; + } + + @Override + public void open(Configuration parameters) throws IOException { + rnd = new Random(); + stringBuilder = new StringBuilder(); + step = getRuntimeContext().getNumberOfParallelSubtasks(); + + + index = getRuntimeContext().getOperatorState("index", getRuntimeContext().getIndexOfThisSubtask(), false); + + isRunning = true; + } + + @Override + public void run(SourceContext ctx) throws Exception { + final Object lockingObject = ctx.getCheckpointLock(); + + while (isRunning && index.value() < numElements) { + char first = (char) ((index.value() % 40) + 40); + + stringBuilder.setLength(0); + stringBuilder.append(first); + + String result = randomString(stringBuilder, rnd); + + synchronized (lockingObject) { + index.update(index.value() + step); +// System.out.println("SOURCE EMIT: " + result); + ctx.collect(result); + } + } + } + + @Override + public void cancel() { + isRunning = false; + } + + private static String randomString(StringBuilder bld, Random rnd) { + final int len = rnd.nextInt(10) + 5; + + for (int i = 0; i < len; i++) { + char next = (char) (rnd.nextInt(20000) + 33); + bld.append(next); + } + + return bld.toString(); + } + } + + private static class StatefulCounterFunction extends RichMapFunction { + + private OperatorState count; + static final long[] counts = new long[PARALLELISM]; + + @Override + public PrefixCount map(PrefixCount value) throws Exception { + count.update(count.value() + 1); + return value; + } + + @Override + public void open(Configuration conf) throws IOException { + count = getRuntimeContext().getOperatorState("count", 0L, false); + } + + @Override + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count.value(); + } + + } + + private static class OnceFailingReducer extends RichReduceFunction { + + private static volatile boolean hasFailed = false; + + private final long numElements; + + private long failurePos; + private long count; + + OnceFailingReducer(long numElements) { + this.numElements = numElements; + } + + @Override + public void open(Configuration parameters) { + long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); + long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); + + failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin; + count = 0; + } + + @Override + public PrefixCount reduce(PrefixCount value1, PrefixCount value2) throws Exception { + count++; + if (!hasFailed && count >= failurePos) { + hasFailed = true; + throw new Exception("Test Failure"); + } + + value1.count += value2.count; + return value1; + } + } + + // -------------------------------------------------------------------------------------------- + // Custom Type Classes + // -------------------------------------------------------------------------------------------- + + public static class PrefixCount { + + public String prefix; + public String value; + public long count; + + public PrefixCount() {} + + public PrefixCount(String prefix, String value, long count) { + this.prefix = prefix; + this.value = value; + this.count = count; + } + + @Override + public String toString() { + return prefix + " / " + value; + } + } + + private static class StringRichFilterFunction extends RichFilterFunction implements Checkpointed { + + Long count = 0L; + static final long[] counts = new long[PARALLELISM]; + + @Override + public boolean filter(String value) { + count++; + return value.length() < 100; + } + + @Override + public void close() { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count; + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + return count; + } + + @Override + public void restoreState(Long state) { + count = state; + } + } + + private static class StringPrefixCountRichMapFunction extends RichMapFunction implements Checkpointed { + + private long count = 0; + static final long[] counts = new long[PARALLELISM]; + static volatile boolean restoreCalledAtLeastOnce = false; + + @Override + public PrefixCount map(String value) throws IOException { + count += 1; + return new PrefixCount(value.substring(0, 1), value, 1L); + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + return count; + } + + @Override + public void restoreState(Long state) { + restoreCalledAtLeastOnce = true; + count = state; + if (count == 0) { + throw new RuntimeException("Restore from beginning"); + } + } + + @Override + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count; + } + } + + private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction implements Checkpointed { + + long count = 0; + static final long[] counts = new long[PARALLELISM]; + + static volatile boolean restoreCalledAtLeastOnce = false; + + @Override + public void flatMap1(String value, Collector out) throws IOException { + count += 1; +// System.out.println("Co-Map COUNT: " + count); + + out.collect(value); + } + + @Override + public void flatMap2(String value, Collector out) throws IOException { + // we ignore the values from the second input + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + return count; + } + + @Override + public void restoreState(Long state) { + restoreCalledAtLeastOnce = true; + count = state; + } + + @Override + public void close() throws IOException { + counts[getRuntimeContext().getIndexOfThisSubtask()] = count; + } + } +}