You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2016/12/20 05:05:37 UTC

[1/7] flink git commit: [FLINK-4391] Add asynchronous I/O operations

Repository: flink
Updated Branches:
  refs/heads/master 4a27d2105 -> bfdaa3821


http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java
new file mode 100644
index 0000000..b8788c6
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java
@@ -0,0 +1,164 @@
+/*
+ * 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.async;
+
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test case for {@link RichAsyncFunction}
+ */
+public class RichAsyncFunctionTest {
+
+	private RichAsyncFunction<String, String> initFunction() {
+		RichAsyncFunction<String, String> function = new RichAsyncFunction<String, String>() {
+			@Override
+			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
+				getRuntimeContext().getState(mock(ValueStateDescriptor.class));
+			}
+		};
+
+		return function;
+	}
+
+	@Test
+	public void testIterationRuntimeContext() throws Exception {
+		// test runtime context is not set
+		RichAsyncFunction<String, String> function = new RichAsyncFunction<String, String>() {
+			@Override
+			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
+				getIterationRuntimeContext().getIterationAggregator("test");
+			}
+		};
+
+		try {
+			function.asyncInvoke("test", mock(AsyncCollector.class));
+		}
+		catch (Exception e) {
+			Assert.assertEquals("The runtime context has not been initialized.", e.getMessage());
+		}
+
+		// test get agg from iteration runtime context
+		function.setRuntimeContext(mock(IterationRuntimeContext.class));
+
+		try {
+			function.asyncInvoke("test", mock(AsyncCollector.class));
+		}
+		catch (Exception e) {
+			Assert.assertEquals("Get iteration aggregator is not supported in rich async function", e.getMessage());
+		}
+
+		// get state from iteration runtime context
+		function = new RichAsyncFunction<String, String>() {
+			@Override
+			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
+				getIterationRuntimeContext().getState(mock(ValueStateDescriptor.class));
+			}
+		};
+
+		function.setRuntimeContext(mock(RuntimeContext.class));
+
+		try {
+			function.asyncInvoke("test", mock(AsyncCollector.class));
+		}
+		catch (Exception e) {
+			Assert.assertEquals("State is not supported in rich async function", e.getMessage());
+		}
+
+		// test getting a counter from iteration runtime context
+		function = new RichAsyncFunction<String, String>() {
+			@Override
+			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
+				getIterationRuntimeContext().getIntCounter("test").add(6);
+			}
+		};
+
+		IterationRuntimeContext context = mock(IterationRuntimeContext.class);
+		IntCounter counter = new IntCounter(0);
+		when(context.getIntCounter(anyString())).thenReturn(counter);
+
+		function.setRuntimeContext(context);
+
+		function.asyncInvoke("test", mock(AsyncCollector.class));
+
+		Assert.assertTrue(6 == counter.getLocalValue());
+	}
+
+	@Test
+	public void testRuntimeContext() throws Exception {
+		// test run time context is not set
+		RichAsyncFunction<String, String> function = new RichAsyncFunction<String, String>() {
+			@Override
+			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
+				getRuntimeContext().getState(mock(ValueStateDescriptor.class));
+			}
+		};
+
+		try {
+			function.asyncInvoke("test", mock(AsyncCollector.class));
+		}
+		catch (Exception e) {
+			Assert.assertEquals("The runtime context has not been initialized.", e.getMessage());
+		}
+
+		// test get state
+		function = new RichAsyncFunction<String, String>() {
+			@Override
+			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
+				getRuntimeContext().getState(mock(ValueStateDescriptor.class));
+			}
+		};
+
+		function.setRuntimeContext(mock(RuntimeContext.class));
+
+		try {
+			function.asyncInvoke("test", mock(AsyncCollector.class));
+		}
+		catch (Exception e) {
+			Assert.assertEquals("State is not supported in rich async function", e.getMessage());
+		}
+
+		// test getting a counter from runtime context
+		function = new RichAsyncFunction<String, String>() {
+			@Override
+			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
+				getIterationRuntimeContext().getIntCounter("test").add(6);
+			}
+		};
+
+		IterationRuntimeContext context = mock(IterationRuntimeContext.class);
+		IntCounter counter = new IntCounter(0);
+		when(context.getIntCounter(anyString())).thenReturn(counter);
+
+		function.setRuntimeContext(context);
+
+		function.asyncInvoke("test", mock(AsyncCollector.class));
+
+		Assert.assertTrue(6 == counter.getLocalValue());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java
new file mode 100644
index 0000000..d118d80
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java
@@ -0,0 +1,656 @@
+/*
+ * 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.async;
+
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
+import org.apache.flink.streaming.api.functions.async.buffer.StreamElementEntry;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.streaming.api.functions.async.buffer.AsyncCollectorBuffer;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link AsyncCollectorBuffer}. These test that:
+ *
+ * <ul>
+ *     <li>Add a new item into the buffer</li>
+ *     <li>Ordered mode processing</li>
+ *     <li>Unordered mode processing</li>
+ *     <li>Error handling</li>
+ * </ul>
+ */
+public class AsyncCollectorBufferTest {
+	private final static ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(10);
+
+	private final Random RANDOM = new Random();
+
+	private AsyncFunction<Integer, Integer> function;
+
+	private AsyncWaitOperator<Integer, Integer> operator;
+
+	private AsyncCollectorBuffer<Integer, Integer> buffer;
+
+	private Output<StreamRecord<Integer>> output;
+
+	private Object lock = new Object();
+
+	public AsyncCollectorBuffer<Integer, Integer> getBuffer(int bufferSize, AsyncDataStream.OutputMode mode) throws Exception {
+		function = new AsyncFunction<Integer, Integer>() {
+			@Override
+			public void asyncInvoke(Integer input, AsyncCollector<Integer> collector) throws Exception {
+
+			}
+		};
+
+		operator = new AsyncWaitOperator<>(function, bufferSize, mode);
+
+		StreamConfig cfg = new StreamConfig(new Configuration());
+		cfg.setTypeSerializerIn1(IntSerializer.INSTANCE);
+
+		StreamTask<?, ?> mockTask = mock(StreamTask.class);
+
+		when(mockTask.getCheckpointLock()).thenReturn(lock);
+
+		Environment env = new DummyEnvironment("DUMMY;-D", 1, 0);
+		when(mockTask.getEnvironment()).thenReturn(env);
+
+		output = new FakedOutput();
+
+		operator.setup(mockTask, cfg, output);
+
+		buffer = operator.getBuffer();
+
+		return buffer;
+	}
+
+	@Test
+	public void testAdd() throws Exception {
+		buffer = getBuffer(3, AsyncDataStream.OutputMode.ORDERED);
+
+		synchronized (lock) {
+			buffer.addWatermark(new Watermark(0l));
+			buffer.addLatencyMarker(new LatencyMarker(111L, 1, 1));
+		}
+
+		Assert.assertEquals(2, buffer.getQueue().size());
+
+		Iterator<StreamElementEntry<Integer>> iterator = buffer.getQueue().iterator();
+		Watermark watermark = iterator.next().getStreamElement().asWatermark();
+		Assert.assertEquals(0l, watermark.getTimestamp());
+
+		LatencyMarker latencyMarker = iterator.next().getStreamElement().asLatencyMarker();
+		Assert.assertEquals(111l, latencyMarker.getMarkedTime());
+
+		buffer.setExtraStreamElement(new Watermark(222l));
+
+		Iterator<StreamElement> elementIterator = buffer.getStreamElementsInBuffer();
+		Assert.assertEquals(0l, elementIterator.next().asWatermark().getTimestamp());
+		Assert.assertEquals(111l, elementIterator.next().asLatencyMarker().getMarkedTime());
+		Assert.assertEquals(222l, elementIterator.next().asWatermark().getTimestamp());
+		Assert.assertFalse(elementIterator.hasNext());
+	}
+
+	private void work(final boolean throwExcept) throws Exception {
+		final int ASYNC_COLLECTOR_NUM = 7;
+
+		Iterator<StreamElement> iterator = new Iterator<StreamElement>() {
+			private int idx = 0;
+
+			@Override
+			public boolean hasNext() {
+				return idx < ASYNC_COLLECTOR_NUM;
+			}
+
+			@Override
+			public StreamElement next() {
+				++idx;
+
+				if (idx == 4) {
+					return new Watermark(333l);
+				}
+				else if (idx == 7) {
+					return new LatencyMarker(111L, 0, 0);
+				}
+				else {
+					StreamRecord<Integer> ret = new StreamRecord<>(idx);
+					ret.setTimestamp(idx * idx);
+
+					return ret;
+				}
+			}
+
+			@Override
+			public void remove() {
+				// do nothing
+			}
+		};
+
+		while (iterator.hasNext()) {
+			final StreamElement record = iterator.next();
+
+			if (record.isRecord()) {
+				AsyncCollector tmp;
+
+				synchronized (lock) {
+					tmp = buffer.addStreamRecord(record.<Integer>asRecord());
+				}
+
+				final AsyncCollector collector = tmp;
+
+				EXECUTOR_SERVICE.submit(new Runnable() {
+					@Override
+					public void run() {
+						try {
+							Thread.sleep(RANDOM.nextInt(100));
+
+							if (throwExcept) {
+								collector.collect(new Exception("wahahahaha..."));
+							}
+							else {
+								collector.collect(Collections.singletonList(record.asRecord().getValue()));
+							}
+						} catch (InterruptedException e) {
+							// do nothing
+						}
+					}
+				});
+			}
+			else if (record.isWatermark()) {
+				synchronized (lock) {
+					buffer.addWatermark(record.asWatermark());
+				}
+			}
+			else {
+				synchronized (lock) {
+					buffer.addLatencyMarker(record.asLatencyMarker());
+				}
+			}
+		}
+	}
+
+	@Test
+	public void testOrderedBuffer() throws Exception {
+		buffer = getBuffer(3, AsyncDataStream.OutputMode.ORDERED);
+
+		buffer.startEmitterThread();
+
+		work(false);
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+		}
+
+		buffer.stopEmitterThread();
+
+		Assert.assertEquals("1,2,3,5,6,", ((FakedOutput)output).getValue());
+		Assert.assertEquals("1,4,9,333,25,36,111,", ((FakedOutput)output).getTimestamp());
+	}
+
+	@Test
+	public void testUnorderedBuffer() throws Exception {
+		buffer = getBuffer(3, AsyncDataStream.OutputMode.UNORDERED);
+
+		buffer.startEmitterThread();
+
+		work(false);
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+		}
+
+		buffer.stopEmitterThread();
+
+		Assert.assertEquals(333L, ((FakedOutput)output).getRawTimestamp().toArray()[3]);
+
+		List<Long> result = ((FakedOutput)output).getRawValue();
+		Collections.sort(result);
+		Assert.assertEquals("[1, 2, 3, 5, 6]", result.toString());
+
+		result = ((FakedOutput)output).getRawTimestamp();
+		Collections.sort(result);
+		Assert.assertEquals("[1, 4, 9, 25, 36, 111, 333]", result.toString());
+	}
+
+	@Test
+	public void testOrderedBufferWithManualTriggering() throws Exception {
+		// test AsyncCollectorBuffer with different combinations of StreamElements in the buffer.
+		// by triggering completion of each AsyncCollector one by one manually, we can verify
+		// the output one by one accurately.
+
+		FakedOutput fakedOutput;
+		AsyncCollector<Integer> collector1, collector2;
+
+		// 1. head element is a Watermark or LatencyMarker
+		buffer = getBuffer(3, AsyncDataStream.OutputMode.ORDERED);
+		fakedOutput = (FakedOutput)output;
+
+		fakedOutput.expect(1);
+
+		buffer.startEmitterThread();
+
+		synchronized (lock) {
+			buffer.addWatermark(new Watermark(1L));
+		}
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("", fakedOutput.getValue());
+		Assert.assertEquals("1,", fakedOutput.getTimestamp());
+
+
+		fakedOutput.expect(1);
+
+		synchronized (lock) {
+			buffer.addLatencyMarker(new LatencyMarker(2L, 0, 0));
+		}
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("", fakedOutput.getValue());
+		Assert.assertEquals("1,2,", fakedOutput.getTimestamp());
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+			buffer.stopEmitterThread();
+		}
+
+
+		// 2. buffer layout: WM -> SR1 -> LM -> SR2, where SR2 finishes first, then SR1.
+		buffer = getBuffer(5, AsyncDataStream.OutputMode.ORDERED);
+		fakedOutput = (FakedOutput)output;
+
+		synchronized (lock) {
+			buffer.addWatermark(new Watermark(1L));
+			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
+			buffer.addLatencyMarker(new LatencyMarker(3L, 0, 0));
+			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 4L));
+		}
+
+		fakedOutput.expect(1);
+
+		buffer.startEmitterThread();
+
+		fakedOutput.waitToFinish();
+
+		// in ORDERED mode, the result of completed SR2 will not be emitted right now.
+		collector2.collect(Collections.singletonList(222));
+
+		Thread.sleep(1000);
+
+		Assert.assertEquals("", fakedOutput.getValue());
+		Assert.assertEquals("1,", fakedOutput.getTimestamp());
+
+		fakedOutput.expect(3);
+
+		collector1.collect(Collections.singletonList(111));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("111,222,", fakedOutput.getValue());
+		Assert.assertEquals("1,2,3,4,", fakedOutput.getTimestamp());
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+			buffer.stopEmitterThread();
+		}
+
+		// 3. buffer layout: WM -> SR1 -> LM -> S2, where SR1 completes first, then SR2.
+		buffer = getBuffer(5, AsyncDataStream.OutputMode.ORDERED);
+		fakedOutput = (FakedOutput)output;
+
+		synchronized (lock) {
+			buffer.addWatermark(new Watermark(1L));
+			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
+			buffer.addLatencyMarker(new LatencyMarker(3L, 0, 0));
+			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 4L));
+		}
+
+		fakedOutput.expect(1);
+
+		buffer.startEmitterThread();
+
+		fakedOutput.waitToFinish();
+
+		fakedOutput.expect(2);
+
+		// in ORDERED mode, the result of completed SR1 will be emitted asap.
+		collector1.collect(Collections.singletonList(111));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("111,", fakedOutput.getValue());
+		Assert.assertEquals("1,2,3,", fakedOutput.getTimestamp());
+
+		fakedOutput.expect(1);
+
+		collector2.collect(Collections.singletonList(222));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("111,222,", fakedOutput.getValue());
+		Assert.assertEquals("1,2,3,4,", fakedOutput.getTimestamp());
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+			buffer.stopEmitterThread();
+		}
+
+		// 4. buffer layout: SR1 -> SR2 -> WM -> LM, where SR2 finishes first.
+		buffer = getBuffer(5, AsyncDataStream.OutputMode.ORDERED);
+		fakedOutput = (FakedOutput)output;
+
+		synchronized (lock) {
+			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 1L));
+			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 2L));
+			buffer.addWatermark(new Watermark(3L));
+			buffer.addLatencyMarker(new LatencyMarker(4L, 0, 0));
+		}
+
+		buffer.startEmitterThread();
+
+		// in ORDERED mode, the result of completed SR2 will not be emitted right now.
+		collector2.collect(Collections.singletonList(222));
+
+		Thread.sleep(1000);
+
+		Assert.assertEquals("", fakedOutput.getValue());
+		Assert.assertEquals("", fakedOutput.getTimestamp());
+
+		fakedOutput.expect(4);
+
+		collector1.collect(Collections.singletonList(111));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("111,222,", fakedOutput.getValue());
+		Assert.assertEquals("1,2,3,4,", fakedOutput.getTimestamp());
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+			buffer.stopEmitterThread();
+		}
+	}
+
+	@Test
+	public void testUnorderedWithManualTriggering() throws Exception {
+		// verify the output in UNORDERED mode by manual triggering.
+
+		FakedOutput fakedOutput;
+		AsyncCollector<Integer> collector1, collector2, collector3;
+
+		// 1. head element is a Watermark or LatencyMarker
+		buffer = getBuffer(5, AsyncDataStream.OutputMode.UNORDERED);
+		fakedOutput = (FakedOutput)output;
+
+		fakedOutput.expect(1);
+
+		buffer.startEmitterThread();
+
+		synchronized (lock) {
+			buffer.addWatermark(new Watermark(1L));
+		}
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("", fakedOutput.getValue());
+		Assert.assertEquals("1,", fakedOutput.getTimestamp());
+
+
+		fakedOutput.expect(1);
+
+		synchronized (lock) {
+			buffer.addLatencyMarker(new LatencyMarker(2L, 0, 0));
+		}
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("", fakedOutput.getValue());
+		Assert.assertEquals("1,2,", fakedOutput.getTimestamp());
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+			buffer.stopEmitterThread();
+		}
+
+
+		// 2. buffer layout: LM -> SR1 -> SR2 -> WM1 -> SR3 -> WM2, where the order of completion is SR3, SR2, SR1
+		buffer = getBuffer(6, AsyncDataStream.OutputMode.UNORDERED);
+		fakedOutput = (FakedOutput)output;
+
+		synchronized (lock) {
+			buffer.addLatencyMarker(new LatencyMarker(1L, 0, 0));
+			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
+			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 3L));
+			buffer.addWatermark(new Watermark(4L));
+			collector3 = buffer.addStreamRecord(new StreamRecord<>(333, 5L));
+			buffer.addWatermark(new Watermark(6L));
+		}
+
+		fakedOutput.expect(1);
+
+		buffer.startEmitterThread();
+
+		fakedOutput.waitToFinish();
+
+		// in UNORDERED mode, the result of completed SR3 will not be emitted right now.
+		collector3.collect(Collections.singletonList(333));
+
+		Thread.sleep(1000);
+
+		Assert.assertEquals("", fakedOutput.getValue());
+		Assert.assertEquals("1,", fakedOutput.getTimestamp());
+
+		fakedOutput.expect(1);
+
+		// SR2 will be emitted
+		collector2.collect(Collections.singletonList(222));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("222,", fakedOutput.getValue());
+		Assert.assertEquals("1,3,", fakedOutput.getTimestamp());
+
+		// SR1 will be emitted first, then WM, and then SR3 and WM2
+		fakedOutput.expect(4);
+		collector1.collect(Collections.singletonList(111));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("222,111,333,", fakedOutput.getValue());
+		Assert.assertEquals("1,3,2,4,5,6,", fakedOutput.getTimestamp());
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+			buffer.stopEmitterThread();
+		}
+
+		// 3. buffer layout: WM1 -> SR1 -> SR2 -> LM -> SR3 -> WM2, where the order of completion is SR2, SR1, SR3
+		buffer = getBuffer(6, AsyncDataStream.OutputMode.UNORDERED);
+		fakedOutput = (FakedOutput)output;
+
+		synchronized (lock) {
+			buffer.addWatermark(new Watermark(1L));
+			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
+			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 3L));
+			buffer.addLatencyMarker(new LatencyMarker(4L, 0, 0));
+			collector3 = buffer.addStreamRecord(new StreamRecord<>(333, 5L));
+			buffer.addWatermark(new Watermark(6L));
+		}
+
+		// the result of SR2 will be emitted following WM1
+		collector2.collect(Collections.singletonList(222));
+
+		fakedOutput.expect(2);
+
+		buffer.startEmitterThread();
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("222,", fakedOutput.getValue());
+		Assert.assertEquals("1,3,", fakedOutput.getTimestamp());
+
+		// SR1 and LM will be emitted
+		fakedOutput.expect(2);
+		collector1.collect(Collections.singletonList(111));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("222,111,", fakedOutput.getValue());
+		Assert.assertEquals("1,3,2,4,", fakedOutput.getTimestamp());
+
+		// SR3 and WM2 will be emitted
+		fakedOutput.expect(2);
+		collector3.collect(Collections.singletonList(333));
+
+		fakedOutput.waitToFinish();
+
+		Assert.assertEquals("222,111,333,", fakedOutput.getValue());
+		Assert.assertEquals("1,3,2,4,5,6,", fakedOutput.getTimestamp());
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+			buffer.stopEmitterThread();
+		}
+
+	}
+
+
+
+	@Test
+	public void testBufferWithException() throws Exception {
+		buffer = getBuffer(3, AsyncDataStream.OutputMode.UNORDERED);
+
+		buffer.startEmitterThread();
+
+		IOException expected = null;
+		try {
+			work(true);
+		}
+		catch (IOException e) {
+			expected = e;
+		}
+
+		Assert.assertNotNull(expected);
+		Assert.assertEquals(expected.getMessage(), "wahahahaha...");
+
+		synchronized (lock) {
+			buffer.waitEmpty();
+		}
+
+		buffer.stopEmitterThread();
+	}
+
+	public class FakedOutput implements Output<StreamRecord<Integer>> {
+		private List<Long> outputs;
+		private List<Long> timestamps;
+
+		private CountDownLatch latch;
+
+		public FakedOutput() {
+			this.outputs = new ArrayList<>();
+			this.timestamps = new ArrayList<>();
+		}
+
+		@Override
+		public void collect(StreamRecord<Integer> record) {
+			outputs.add(record.getValue().longValue());
+			if (record.hasTimestamp()) {
+				timestamps.add(record.getTimestamp());
+			}
+
+			if (latch != null) {
+				latch.countDown();
+			}
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+			timestamps.add(mark.getTimestamp());
+
+			if (latch != null) {
+				latch.countDown();
+			}
+		}
+
+		@Override
+		public void emitLatencyMarker(LatencyMarker latencyMarker) {
+			timestamps.add(latencyMarker.getMarkedTime());
+
+			if (latch != null) {
+				latch.countDown();
+			}
+		}
+
+		@Override
+		public void close() {
+		}
+
+		public String getValue() {
+			StringBuilder sb = new StringBuilder();
+			for (Long i : outputs) {
+				sb.append(i).append(",");
+			}
+			return sb.toString();
+		}
+
+		public String getTimestamp() {
+			StringBuilder sb = new StringBuilder();
+			for (Long i : timestamps) {
+				sb.append(i).append(",");
+			}
+			return sb.toString();
+		}
+
+		public List<Long> getRawValue() {
+			return outputs;
+		}
+
+		public List<Long> getRawTimestamp() {
+			return timestamps;
+		}
+
+		public void expect(int count) {
+			latch = new CountDownLatch(count);
+		}
+
+		public void waitToFinish() throws InterruptedException {
+			latch.await();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
new file mode 100644
index 0000000..560ee5a
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
@@ -0,0 +1,629 @@
+/*
+ * 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.async;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.SubtaskState;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
+import org.apache.flink.runtime.state.TaskStateHandles;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.async.RichAsyncFunction;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
+import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Queue;
+import java.util.Random;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link AsyncWaitOperator}. These test that:
+ *
+ * <ul>
+ *     <li>Process StreamRecords and Watermarks in ORDERED mode</li>
+ *     <li>Process StreamRecords and Watermarks in UNORDERED mode</li>
+ *     <li>AsyncWaitOperator in operator chain</li>
+ *     <li>Snapshot state and restore state</li>
+ * </ul>
+ */
+public class AsyncWaitOperatorTest {
+
+	// hold sink result
+	private static Queue<Object> sinkResult;
+
+	private static class MyAsyncFunction extends RichAsyncFunction<Integer, Integer> {
+		final int SLEEP_FACTOR = 100;
+		final int THREAD_POOL_SIZE = 10;
+
+		transient static ExecutorService executorService;
+		static int counter = 0;
+
+		static Random random = new Random();
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			synchronized (MyAsyncFunction.class) {
+				if (counter == 0) {
+					executorService = Executors.newFixedThreadPool(THREAD_POOL_SIZE);
+				}
+
+				++counter;
+			}
+		}
+
+		@Override
+		public void close() throws Exception {
+			super.close();
+
+			synchronized (MyAsyncFunction.class) {
+				--counter;
+
+				if (counter == 0) {
+					executorService.shutdown();
+					executorService.awaitTermination(SLEEP_FACTOR * THREAD_POOL_SIZE, TimeUnit.MILLISECONDS);
+				}
+			}
+		}
+
+		@Override
+		public void asyncInvoke(final Integer input, final AsyncCollector<Integer> collector) throws Exception {
+			this.executorService.submit(new Runnable() {
+				@Override
+				public void run() {
+					// wait for while to simulate async operation here
+					int sleep = (int) (random.nextFloat() * SLEEP_FACTOR);
+
+					try {
+						Thread.sleep(sleep);
+						List<Integer> ret = new ArrayList<>();
+						ret.add(input*2);
+						collector.collect(ret);
+					}
+					catch (InterruptedException e) {
+						// do nothing
+					}
+				}
+			});
+		}
+	}
+
+	/**
+	 * A special {@link org.apache.flink.streaming.api.functions.async.AsyncFunction} without issuing
+	 * {@link AsyncCollector#collect} until the latch counts to zero.
+	 * This function is used in the testStateSnapshotAndRestore, ensuring
+	 * that {@link org.apache.flink.streaming.api.functions.async.buffer.StreamElementEntry} can stay
+	 * in the {@link org.apache.flink.streaming.api.functions.async.buffer.AsyncCollectorBuffer} to be
+	 * snapshotted while checkpointing.
+	 */
+	private static class LazyAsyncFunction extends MyAsyncFunction {
+		private static CountDownLatch latch;
+
+		public LazyAsyncFunction() {
+			latch = new CountDownLatch(1);
+		}
+
+		@Override
+		public void asyncInvoke(final Integer input, final AsyncCollector<Integer> collector) throws Exception {
+			this.executorService.submit(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						latch.await();
+					}
+					catch (InterruptedException e) {
+						// do nothing
+					}
+
+					collector.collect(Collections.singletonList(input));
+				}
+			});
+		}
+
+		public static void countDown() {
+			latch.countDown();
+		}
+	}
+
+	/**
+	 * A {@link Comparator} to compare {@link StreamRecord} while sorting them.
+	 */
+	private class StreamRecordComparator implements Comparator<Object> {
+		@Override
+		public int compare(Object o1, Object o2) {
+			if (o1 instanceof Watermark || o2 instanceof Watermark) {
+				return 0;
+			} else {
+				StreamRecord<Integer> sr0 = (StreamRecord<Integer>) o1;
+				StreamRecord<Integer> sr1 = (StreamRecord<Integer>) o2;
+
+				if (sr0.getTimestamp() != sr1.getTimestamp()) {
+					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
+				}
+
+				int comparison = sr0.getValue().compareTo(sr1.getValue());
+				if (comparison != 0) {
+					return comparison;
+				} else {
+					return sr0.getValue() - sr1.getValue();
+				}
+			}
+		}
+	}
+
+	@Test
+	public void testWaterMarkOrdered() throws Exception {
+		testWithWatermark(AsyncDataStream.OutputMode.ORDERED);
+	}
+
+	@Test
+	public void testWaterMarkUnordered() throws Exception {
+		testWithWatermark(AsyncDataStream.OutputMode.UNORDERED);
+	}
+
+	private void testWithWatermark(AsyncDataStream.OutputMode mode) throws Exception {
+		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(new MyAsyncFunction(), 2, mode);
+
+		final OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
+				new OneInputStreamOperatorTestHarness<>(operator, IntSerializer.INSTANCE);
+
+		final long initialTime = 0L;
+		final ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		synchronized (testHarness.getCheckpointLock()) {
+			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));
+		}
+
+		// wait until all async collectors in the buffer have been emitted out.
+		synchronized (testHarness.getCheckpointLock()) {
+			testHarness.close();
+		}
+
+		expectedOutput.add(new StreamRecord<>(2, initialTime + 1));
+		expectedOutput.add(new StreamRecord<>(4, initialTime + 2));
+		expectedOutput.add(new Watermark(initialTime + 2));
+		expectedOutput.add(new StreamRecord<>(6, initialTime + 3));
+
+		if (AsyncDataStream.OutputMode.ORDERED == mode) {
+			TestHarnessUtil.assertOutputEquals("Output with watermark was not correct.", expectedOutput, testHarness.getOutput());
+		}
+		else {
+			Object[] jobOutputQueue = testHarness.getOutput().toArray();
+
+			Assert.assertEquals("Watermark should be at index 2", new Watermark(initialTime + 2), jobOutputQueue[2]);
+			Assert.assertEquals("StreamRecord 3 should be at the end", new StreamRecord<>(6, initialTime + 3), jobOutputQueue[3]);
+
+			TestHarnessUtil.assertOutputEqualsSorted(
+					"Output for StreamRecords does not match",
+					expectedOutput,
+					testHarness.getOutput(),
+					new StreamRecordComparator());
+		}
+	}
+
+	@Test
+	public void testOrdered() throws Exception {
+		testRun(AsyncDataStream.OutputMode.ORDERED);
+	}
+
+	@Test
+	public void testUnordered() throws Exception {
+		testRun(AsyncDataStream.OutputMode.UNORDERED);
+	}
+
+	private void testRun(AsyncDataStream.OutputMode mode) throws Exception {
+		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
+		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
+				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
+
+		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(new MyAsyncFunction(), 6, mode);
+
+		final StreamConfig streamConfig = testHarness.getStreamConfig();
+		streamConfig.setStreamOperator(operator);
+
+		testHarness.invoke();
+		testHarness.waitForTaskRunning();
+
+		final long initialTime = 0L;
+		final ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
+
+		testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
+		testHarness.processElement(new StreamRecord<>(2, 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 + 1));
+		expectedOutput.add(new StreamRecord<>(4, initialTime + 2));
+		expectedOutput.add(new StreamRecord<>(6, initialTime + 3));
+		expectedOutput.add(new StreamRecord<>(8, initialTime + 4));
+		expectedOutput.add(new StreamRecord<>(10, initialTime + 5));
+		expectedOutput.add(new StreamRecord<>(12, initialTime + 6));
+		expectedOutput.add(new StreamRecord<>(14, initialTime + 7));
+		expectedOutput.add(new StreamRecord<>(16, initialTime + 8));
+
+		testHarness.waitForInputProcessing();
+
+		testHarness.endInput();
+
+		testHarness.waitForTaskCompletion();
+
+		if (mode == AsyncDataStream.OutputMode.ORDERED) {
+			TestHarnessUtil.assertOutputEquals("ORDERED Output was not correct.", expectedOutput, testHarness.getOutput());
+		}
+		else {
+			TestHarnessUtil.assertOutputEqualsSorted(
+					"UNORDERED Output was not correct.",
+					expectedOutput,
+					testHarness.getOutput(),
+					new StreamRecordComparator());
+		}
+	}
+
+	private JobVertex createChainedVertex(boolean withLazyFunction) {
+		StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// the input is only used to construct a chained operator, and they will not be used in the real tests.
+		DataStream<Integer> input = chainEnv.fromElements(1, 2, 3);
+
+		if (withLazyFunction) {
+			input = AsyncDataStream.orderedWait(input, new LazyAsyncFunction(), 6);
+		}
+		else {
+			input = AsyncDataStream.orderedWait(input, new MyAsyncFunction(), 6);
+		}
+
+		// the map function is designed to chain after async function. we place an Integer object in it and
+		// it is initialized in the open() method.
+		// it is used to verify that operators in the operator chain should be opened from the tail to the head,
+		// so the result from AsyncWaitOperator can pass down successfully and correctly.
+		// if not, the test can not be passed.
+		input = input.map(new RichMapFunction<Integer, Integer>() {
+			private static final long serialVersionUID = 1L;
+
+			private Integer initialValue = null;
+
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				initialValue = 1;
+			}
+
+			@Override
+			public Integer map(Integer value) throws Exception {
+				return initialValue + value;
+			}
+		});
+
+		input = AsyncDataStream.unorderedWait(input, new MyAsyncFunction(), 3);
+
+		input.addSink(new SinkFunction<Integer>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void invoke(Integer value) throws Exception {
+				sinkResult.add(value);
+			}
+		});
+
+		// be build our own OperatorChain
+		final JobGraph jobGraph = chainEnv.getStreamGraph().getJobGraph();
+
+		Assert.assertTrue(jobGraph.getVerticesSortedTopologicallyFromSources().size() == 2);
+
+		return jobGraph.getVerticesSortedTopologicallyFromSources().get(1);
+	}
+
+	/**
+	 * Get the {@link SubtaskState} for the operator chain. The state will keep several inputs.
+	 *
+	 * @return A {@link SubtaskState}
+	 * @throws Exception
+     */
+	private SubtaskState createTaskState() throws Exception {
+		sinkResult = new ConcurrentLinkedDeque<>();
+
+		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
+		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
+				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
+
+		JobVertex chainedVertex = createChainedVertex(true);
+
+		testHarness.taskConfig = chainedVertex.getConfiguration();
+
+		final AcknowledgeStreamMockEnvironment env = new AcknowledgeStreamMockEnvironment(
+				testHarness.jobConfig,
+				testHarness.taskConfig,
+				testHarness.getExecutionConfig(),
+				testHarness.memorySize,
+				new MockInputSplitProvider(),
+				testHarness.bufferSize);
+
+		final StreamConfig streamConfig = testHarness.getStreamConfig();
+		final StreamConfig operatorChainStreamConfig = new StreamConfig(chainedVertex.getConfiguration());
+		final AsyncWaitOperator<Integer, Integer> headOperator =
+				operatorChainStreamConfig.getStreamOperator(Thread.currentThread().getContextClassLoader());
+		streamConfig.setStreamOperator(headOperator);
+
+		testHarness.invoke(env);
+		testHarness.waitForTaskRunning();
+
+		testHarness.processElement(new StreamRecord<>(1));
+		testHarness.processElement(new StreamRecord<>(2));
+		testHarness.processElement(new StreamRecord<>(3));
+		testHarness.processElement(new StreamRecord<>(4));
+
+		testHarness.waitForInputProcessing();
+
+		final CheckpointMetaData checkpointMetaData = new CheckpointMetaData(1L, 1L);
+
+		task.triggerCheckpoint(checkpointMetaData);
+
+		env.getCheckpointLatch().await();
+
+		assertEquals(1L, env.getCheckpointId());
+
+		LazyAsyncFunction.countDown();
+
+		testHarness.endInput();
+		testHarness.waitForTaskCompletion();
+
+		return env.getCheckpointStateHandles();
+	}
+
+	@Test
+	public void testOperatorChain() throws Exception {
+
+		JobVertex chainedVertex = createChainedVertex(false);
+
+		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
+		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
+				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
+
+		task.setInitialState(new TaskStateHandles(createTaskState()));
+
+		sinkResult = new ConcurrentLinkedDeque<>();
+
+		testHarness.taskConfig = chainedVertex.getConfiguration();
+
+		final AcknowledgeStreamMockEnvironment env = new AcknowledgeStreamMockEnvironment(
+				testHarness.jobConfig,
+				testHarness.taskConfig,
+				testHarness.getExecutionConfig(),
+				testHarness.memorySize,
+				new MockInputSplitProvider(),
+				testHarness.bufferSize);
+
+		final StreamConfig streamConfig = testHarness.getStreamConfig();
+		final StreamConfig operatorChainStreamConfig = new StreamConfig(chainedVertex.getConfiguration());
+		final AsyncWaitOperator<Integer, Integer> headOperator =
+				operatorChainStreamConfig.getStreamOperator(Thread.currentThread().getContextClassLoader());
+		streamConfig.setStreamOperator(headOperator);
+
+		testHarness.invoke(env);
+		testHarness.waitForTaskRunning();
+
+		testHarness.processElement(new StreamRecord<>(5));
+		testHarness.processElement(new StreamRecord<>(6));
+		testHarness.processElement(new StreamRecord<>(7));
+		testHarness.processElement(new StreamRecord<>(8));
+		testHarness.processElement(new StreamRecord<>(9));
+
+		testHarness.endInput();
+		testHarness.waitForTaskCompletion();
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+		expectedOutput.add(6);
+		expectedOutput.add(10);
+		expectedOutput.add(14);
+		expectedOutput.add(18);
+		expectedOutput.add(22);
+		expectedOutput.add(26);
+		expectedOutput.add(30);
+		expectedOutput.add(34);
+		expectedOutput.add(38);
+
+		TestHarnessUtil.assertOutputEqualsSorted(
+				"Test for chained operator with AsyncWaitOperator failed",
+				expectedOutput,
+				sinkResult,
+				new Comparator<Object>() {
+					@Override
+					public int compare(Object o1, Object o2) {
+						return (Integer)o1 - (Integer)o2;
+					}
+				});
+	}
+
+	@Test
+	public void testStateSnapshotAndRestore() throws Exception {
+		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
+		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
+				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
+
+		AsyncWaitOperator<Integer, Integer> operator =
+				new AsyncWaitOperator<>(new LazyAsyncFunction(), 6, AsyncDataStream.OutputMode.ORDERED);
+
+		final StreamConfig streamConfig = testHarness.getStreamConfig();
+		streamConfig.setStreamOperator(operator);
+
+		final AcknowledgeStreamMockEnvironment env = new AcknowledgeStreamMockEnvironment(
+				testHarness.jobConfig,
+				testHarness.taskConfig,
+				testHarness.getExecutionConfig(),
+				testHarness.memorySize,
+				new MockInputSplitProvider(),
+				testHarness.bufferSize);
+
+		testHarness.invoke(env);
+		testHarness.waitForTaskRunning();
+
+		final long initialTime = 0L;
+
+		testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
+		testHarness.processElement(new StreamRecord<>(2, initialTime + 2));
+		testHarness.processElement(new StreamRecord<>(3, initialTime + 3));
+		testHarness.processElement(new StreamRecord<>(4, initialTime + 4));
+
+		testHarness.waitForInputProcessing();
+
+		final long checkpointId = 1L;
+		final long checkpointTimestamp = 1L;
+
+		final CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, checkpointTimestamp);
+
+		task.triggerCheckpoint(checkpointMetaData);
+
+		env.getCheckpointLatch().await();
+
+		assertEquals(checkpointId, env.getCheckpointId());
+
+		LazyAsyncFunction.countDown();
+
+		testHarness.endInput();
+		testHarness.waitForTaskCompletion();
+
+		// set the operator state from previous attempt into the restored one
+		final OneInputStreamTask<Integer, Integer> restoredTask = new OneInputStreamTask<>();
+		restoredTask.setInitialState(new TaskStateHandles(env.getCheckpointStateHandles()));
+
+		final OneInputStreamTaskTestHarness<Integer, Integer> restoredTaskHarness =
+				new OneInputStreamTaskTestHarness<>(restoredTask, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
+
+		AsyncWaitOperator<Integer, Integer> restoredOperator =
+				new AsyncWaitOperator<>(new MyAsyncFunction(), 6, AsyncDataStream.OutputMode.ORDERED);
+
+		restoredTaskHarness.getStreamConfig().setStreamOperator(restoredOperator);
+
+		restoredTaskHarness.invoke();
+		restoredTaskHarness.waitForTaskRunning();
+
+		restoredTaskHarness.processElement(new StreamRecord<>(5, initialTime + 5));
+		restoredTaskHarness.processElement(new StreamRecord<>(6, initialTime + 6));
+		restoredTaskHarness.processElement(new StreamRecord<>(7, initialTime + 7));
+
+		// trigger the checkpoint while processing stream elements
+		restoredTask.triggerCheckpoint(new CheckpointMetaData(checkpointId, checkpointTimestamp));
+
+		restoredTaskHarness.processElement(new StreamRecord<>(8, initialTime + 8));
+
+		restoredTaskHarness.endInput();
+		restoredTaskHarness.waitForTaskCompletion();
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+		expectedOutput.add(new StreamRecord<>(2, initialTime + 1));
+		expectedOutput.add(new StreamRecord<>(4, initialTime + 2));
+		expectedOutput.add(new StreamRecord<>(6, initialTime + 3));
+		expectedOutput.add(new StreamRecord<>(8, initialTime + 4));
+		expectedOutput.add(new StreamRecord<>(10, initialTime + 5));
+		expectedOutput.add(new StreamRecord<>(12, initialTime + 6));
+		expectedOutput.add(new StreamRecord<>(14, initialTime + 7));
+		expectedOutput.add(new StreamRecord<>(16, initialTime + 8));
+
+		// remove CheckpointBarrier which is not expected
+		Iterator<Object> iterator = restoredTaskHarness.getOutput().iterator();
+		while (iterator.hasNext()) {
+			if (iterator.next() instanceof CheckpointBarrier) {
+				iterator.remove();
+			}
+		}
+
+		TestHarnessUtil.assertOutputEquals(
+				"StateAndRestored Test Output was not correct.",
+				expectedOutput,
+				restoredTaskHarness.getOutput());
+	}
+
+	private static class AcknowledgeStreamMockEnvironment extends StreamMockEnvironment {
+		private volatile long checkpointId;
+		private volatile SubtaskState checkpointStateHandles;
+
+		private final OneShotLatch checkpointLatch = new OneShotLatch();
+
+		public long getCheckpointId() {
+			return checkpointId;
+		}
+
+		AcknowledgeStreamMockEnvironment(
+				Configuration jobConfig, Configuration taskConfig,
+				ExecutionConfig executionConfig, long memorySize,
+				MockInputSplitProvider inputSplitProvider, int bufferSize) {
+				super(jobConfig, taskConfig, executionConfig, memorySize, inputSplitProvider, bufferSize);
+		}
+
+
+		@Override
+		public void acknowledgeCheckpoint(
+				CheckpointMetaData checkpointMetaData,
+				SubtaskState checkpointStateHandles) {
+
+			this.checkpointId = checkpointMetaData.getCheckpointId();
+			this.checkpointStateHandles = checkpointStateHandles;
+			checkpointLatch.trigger();
+		}
+
+		public OneShotLatch getCheckpointLatch() {
+			return checkpointLatch;
+		}
+
+		public SubtaskState getCheckpointStateHandles() {
+			return checkpointStateHandles;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java
index c95a85e..4e405fd 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java
@@ -142,6 +142,8 @@ public class StreamOperatorChainingTest {
 
 		StreamConfig streamConfig = new StreamConfig(configuration);
 
+		System.out.println(streamConfig);
+
 		StreamMap<Integer, Integer> headOperator =
 				streamConfig.getStreamOperator(Thread.currentThread().getContextClassLoader());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index b20b3a3..8dc6afa 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -264,6 +264,10 @@ public class StreamTaskTestHarness<OUT> {
 		return streamConfig;
 	}
 
+	public ExecutionConfig getExecutionConfig() {
+		return executionConfig;
+	}
+
 	private void shutdownIOManager() throws Exception {
 		this.mockEnv.getIOManager().shutdown();
 		Assert.assertTrue("IO Manager has not properly shut down.", this.mockEnv.getIOManager().isProperlyShutDown());

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index 7468d9a..86fbaa0 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -17,9 +17,11 @@
  */
 package org.apache.flink.streaming.util;
 
+import org.apache.flink.api.common.typeutils.TypeSerializer;
 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.util.Preconditions;
 
 import java.util.Collection;
 
@@ -35,6 +37,14 @@ public class OneInputStreamOperatorTestHarness<IN, OUT>
 
 	private final OneInputStreamOperator<IN, OUT> oneInputOperator;
 
+	public OneInputStreamOperatorTestHarness(
+			OneInputStreamOperator<IN, OUT> operator,
+			TypeSerializer<IN> typeSerializerIn) throws Exception {
+		this(operator, 1, 1, 0);
+
+		config.setTypeSerializerIn1(Preconditions.checkNotNull(typeSerializerIn));
+	}
+
 	public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator) throws Exception {
 		this(operator, 1, 1, 0);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
index 3b98d33..ea99fe3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
@@ -22,47 +22,27 @@ import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.util.MathUtils;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.functions.async.RichAsyncFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.MathUtils;
+import org.junit.*;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.util.ArrayList;
-import java.util.List;
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 
 public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase {
 
-	private String resultPath1;
-	private String resultPath2;
-	private String expected1;
-	private String expected2;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws Exception {
-		resultPath1 = tempFolder.newFile().toURI().toString();
-		resultPath2 = tempFolder.newFile().toURI().toString();
-		expected1 = "";
-		expected2 = "";
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expected1, resultPath1);
-		compareResultsByLinesInMemory(expected2, resultPath2);
-	}
 
 	/**
 	 * Tests the proper functioning of the streaming fold operator. For this purpose, a stream
@@ -112,6 +92,8 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 				}
 			});
 
+		final MemorySinkFunction sinkFunction1 = new MemorySinkFunction(0);
+
 		splittedResult.select("0").map(new MapFunction<Tuple2<Integer,Integer>, Integer>() {
 			private static final long serialVersionUID = 2114608668010092995L;
 
@@ -119,7 +101,10 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 			public Integer map(Tuple2<Integer, Integer> value) throws Exception {
 				return value.f1;
 			}
-		}).writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
+		}).addSink(sinkFunction1);
+
+
+		final MemorySinkFunction sinkFunction2 = new MemorySinkFunction(1);
 
 		splittedResult.select("1").map(new MapFunction<Tuple2<Integer, Integer>, Integer>() {
 			private static final long serialVersionUID = 5631104389744681308L;
@@ -128,27 +113,34 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 			public Integer map(Tuple2<Integer, Integer> value) throws Exception {
 				return value.f1;
 			}
-		}).writeAsText(resultPath2, FileSystem.WriteMode.OVERWRITE);
+		}).addSink(sinkFunction2);
 
-		StringBuilder builder1 = new StringBuilder();
-		StringBuilder builder2 = new StringBuilder();
+		Collection<Integer> expected1 = new ArrayList<>(10);
+		Collection<Integer> expected2 = new ArrayList<>(10);
 		int counter1 = 0;
 		int counter2 = 0;
 
 		for (int i = 0; i < numElements; i++) {
 			if (MathUtils.murmurHash(i) % numKeys == 0) {
 				counter1 += i;
-				builder1.append(counter1 + "\n");
+				expected1.add(counter1);
 			} else {
 				counter2 += i;
-				builder2.append(counter2 + "\n");
+				expected2.add(counter2);
 			}
 		}
 
-		expected1 = builder1.toString();
-		expected2 = builder2.toString();
-
 		env.execute();
+
+		Collection<Integer> result1 = sinkFunction1.getResult();
+		Collections.sort((ArrayList)result1);
+		Collection<Integer> result2 = sinkFunction2.getResult();
+		Collections.sort((ArrayList)result2);
+
+		Assert.assertArrayEquals(result1.toArray(), expected1.toArray());
+		Assert.assertArrayEquals(result2.toArray(), expected2.toArray());
+
+		MemorySinkFunction.clear();
 	}
 
 	/**
@@ -162,6 +154,8 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		DataStream<Tuple2<Integer, NonSerializable>> input = env.addSource(new NonSerializableTupleSource(numElements));
 
+		final MemorySinkFunction sinkFunction = new MemorySinkFunction(0);
+
 		input
 			.keyBy(0)
 			.fold(
@@ -182,17 +176,100 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 					return value.value;
 				}
 			})
-			.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
+			.addSink(sinkFunction);
 
-		StringBuilder builder = new StringBuilder();
+		Collection<Integer> expected = new ArrayList<>(10);
 
 		for (int i = 0; i < numElements; i++) {
-			builder.append(42 + i + "\n");
+			expected.add(42 + i );
 		}
 
-		expected1 = builder.toString();
+		env.execute();
+
+		Collection<Integer> result = sinkFunction.getResult();
+		Collections.sort((ArrayList)result);
+
+		Assert.assertArrayEquals(result.toArray(), expected.toArray());
+
+		MemorySinkFunction.clear();
+	}
+
+	@Test
+	public void testAsyncWaitOperator() throws Exception {
+		final int numElements = 10;
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<Integer, NonSerializable>> input = env.addSource(new NonSerializableTupleSource(numElements));
+
+		AsyncFunction<Tuple2<Integer, NonSerializable>, Integer> function = new RichAsyncFunction<Tuple2<Integer, NonSerializable>, Integer>() {
+			transient ExecutorService executorService;
+
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				super.open(parameters);
+				executorService = Executors.newFixedThreadPool(numElements);
+			}
+
+			@Override
+			public void close() throws Exception {
+				super.close();
+				executorService.shutdown();
+			}
+
+			@Override
+			public void asyncInvoke(final Tuple2<Integer, NonSerializable> input,
+									final AsyncCollector<Integer> collector) throws Exception {
+				this.executorService.submit(new Runnable() {
+					@Override
+					public void run() {
+						// wait for while to simulate async operation here
+						int sleep = (int) (new Random().nextFloat() * 10);
+						try {
+							Thread.sleep(sleep);
+							List<Integer> ret = new ArrayList<>();
+							ret.add(input.f0+input.f0);
+							collector.collect(ret);
+						}
+						catch (InterruptedException e) {
+							collector.collect(new ArrayList<Integer>(0));
+						}
+					}
+				});
+			}
+		};
+
+		DataStream<Integer> orderedResult = AsyncDataStream.orderedWait(input, function, 2).setParallelism(1);
+
+		// save result from ordered process
+		final MemorySinkFunction sinkFunction1 = new MemorySinkFunction(0);
+
+		orderedResult.addSink(sinkFunction1).setParallelism(1);
+
+
+		DataStream<Integer> unorderedResult = AsyncDataStream.unorderedWait(input, function, 2);
+
+		// save result from unordered process
+		final MemorySinkFunction sinkFunction2 = new MemorySinkFunction(1);
+
+		unorderedResult.addSink(sinkFunction2);
+
+
+		Collection<Integer> expected = new ArrayList<>(10);
+
+		for (int i = 0; i < numElements; i++) {
+			expected.add(i+i);
+		}
 
 		env.execute();
+
+		Assert.assertArrayEquals(expected.toArray(), sinkFunction1.getResult().toArray());
+
+		Collection<Integer> result = sinkFunction2.getResult();
+		Collections.sort((ArrayList)result);
+		Assert.assertArrayEquals(expected.toArray(), result.toArray());
+
+		MemorySinkFunction.clear();
 	}
 
 	private static class NonSerializable {
@@ -247,7 +324,50 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 		}
 
 		@Override
+
+
 		public void cancel() {
 		}
 	}
+
+	private static class MemorySinkFunction implements SinkFunction<Integer> {
+		private final static Collection<Integer> collection1 = new ArrayList<>(10);
+
+		private final static Collection<Integer> collection2 = new ArrayList<>(10);
+
+		private  final long serialVersionUID = -8815570195074103860L;
+
+		private final int idx;
+
+		public MemorySinkFunction(int idx) {
+			this.idx = idx;
+		}
+
+		@Override
+		public void invoke(Integer value) throws Exception {
+			if (idx == 0) {
+				synchronized (collection1) {
+					collection1.add(value);
+				}
+			}
+			else {
+				synchronized (collection2) {
+					collection2.add(value);
+				}
+			}
+		}
+
+		public Collection<Integer> getResult() {
+			if (idx == 0) {
+				return collection1;
+			}
+
+			return collection2;
+		}
+
+		public static void clear() {
+			collection1.clear();
+			collection2.clear();
+		}
+	}
 }


[7/7] flink git commit: [FLINK-4391] Add Scala API for asynchronous I/O operations

Posted by tr...@apache.org.
[FLINK-4391] Add Scala API for asynchronous I/O operations

This commit also adds a small example for asynchronous I/O with Scala.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/bfdaa382
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/bfdaa382
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/bfdaa382

Branch: refs/heads/master
Commit: bfdaa3821c71f9fa3a3ff85f56154995d98b18b5
Parents: 6c5a871
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Dec 20 04:56:38 2016 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Dec 20 05:29:03 2016 +0100

----------------------------------------------------------------------
 .../examples/async/AsyncIOExample.java          |   2 +-
 .../scala/examples/async/AsyncIOExample.scala   |  71 +++++
 .../async/collector/AsyncCollector.java         |   3 -
 .../streaming/api/operators/async/Emitter.java  |   6 +-
 .../streaming/api/scala/AsyncDataStream.scala   | 298 +++++++++++++++++++
 .../api/scala/async/AsyncCollector.scala        |  50 ++++
 .../api/scala/async/AsyncFunction.scala         |  48 +++
 .../scala/async/JavaAsyncCollectorWrapper.scala |  43 +++
 8 files changed, 515 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
index 2b05983..9b1f78f 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
@@ -93,7 +93,7 @@ public class AsyncIOExample {
 						start = 0;
 					}
 				}
-				Thread.sleep(10);
+				Thread.sleep(10L);
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/async/AsyncIOExample.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/async/AsyncIOExample.scala b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/async/AsyncIOExample.scala
new file mode 100644
index 0000000..69c4c0a
--- /dev/null
+++ b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/async/AsyncIOExample.scala
@@ -0,0 +1,71 @@
+/*
+ * 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.scala.examples.async
+
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
+import org.apache.flink.streaming.api.scala._
+import org.apache.flink.streaming.api.scala.async.AsyncCollector
+
+import scala.concurrent.{ExecutionContext, Future}
+
+object AsyncIOExample {
+
+  def main(args: Array[String]) {
+    val timeout = 10000L
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val input = env.addSource(new SimpleSource())
+
+    val asyncMapped = AsyncDataStream.orderedWait(input, timeout, TimeUnit.MILLISECONDS, 10) {
+      (input, collector: AsyncCollector[Int]) =>
+        Future {
+          collector.collect(Seq(input))
+        } (ExecutionContext.global)
+    }
+
+    asyncMapped.print()
+
+    env.execute("Async I/O job")
+  }
+}
+
+class SimpleSource extends ParallelSourceFunction[Int] {
+  var running = true
+  var counter = 0
+
+  override def run(ctx: SourceContext[Int]): Unit = {
+    while (running) {
+      ctx.getCheckpointLock.synchronized {
+        ctx.collect(counter)
+      }
+      counter += 1
+
+      Thread.sleep(10L)
+    }
+  }
+
+  override def cancel(): Unit = {
+    running = false
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
index a072aca..25078ae 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
@@ -37,9 +37,6 @@ public interface AsyncCollector<OUT> {
 	 * <p>
 	 * Put all results in a {@link Collection} and then issue
 	 * {@link AsyncCollector#collect(Collection)}.
-	 * <p>
-	 * If the result is NULL, it will cause task fail. If collecting empty result set is allowable and
-	 * should not cause task fail-over, then try to collect an empty list collection.
 	 *
 	 * @param result A list of results.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
index c122d23..a07abe1 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
@@ -136,8 +136,10 @@ public class Emitter<OUT> implements Runnable {
 				try {
 					Collection<OUT> resultCollection = streamRecordResult.get();
 
-					for (OUT result : resultCollection) {
-						timestampedCollector.collect(result);
+					if (resultCollection != null) {
+						for (OUT result : resultCollection) {
+							timestampedCollector.collect(result);
+						}
 					}
 				} catch (Exception e) {
 					operatorActions.failOperator(

http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AsyncDataStream.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AsyncDataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AsyncDataStream.scala
new file mode 100644
index 0000000..67af484
--- /dev/null
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AsyncDataStream.scala
@@ -0,0 +1,298 @@
+/*
+ * 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.scala
+
+import org.apache.flink.annotation.PublicEvolving
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.streaming.api.datastream.{AsyncDataStream => JavaAsyncDataStream}
+import org.apache.flink.streaming.api.functions.async.collector.{AsyncCollector => JavaAsyncCollector}
+import org.apache.flink.streaming.api.functions.async.{AsyncFunction => JavaAsyncFunction}
+import org.apache.flink.streaming.api.scala.async.{AsyncCollector, AsyncFunction, JavaAsyncCollectorWrapper}
+import org.apache.flink.util.Preconditions
+
+import scala.concurrent.duration.TimeUnit
+
+/**
+  * A helper class to apply [[AsyncFunction]] to a data stream.
+  *
+  * Example:
+  * {{{
+  *   val input: DataStream[String] = ...
+  *   val asyncFunction: (String, AsyncCollector[String]) => Unit = ...
+  *
+  *   AsyncDataStream.orderedWait(input, asyncFunction, timeout, TimeUnit.MILLISECONDS, 100)
+  * }}}
+  */
+@PublicEvolving
+object AsyncDataStream {
+
+  private val DEFAULT_QUEUE_CAPACITY = 100
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is only maintained
+    * with respect to watermarks. Stream records which lie between the same two watermarks, can be
+    * re-ordered.
+    *
+    * @param input to apply the async function on
+    * @param asyncFunction to use
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @param capacity of the operator which is equivalent to the number of concurrent asynchronous
+    *                 operations
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def unorderedWait[IN, OUT: TypeInformation](
+      input: DataStream[IN],
+      asyncFunction: AsyncFunction[IN, OUT],
+      timeout: Long,
+      timeUnit: TimeUnit,
+      capacity: Int)
+    : DataStream[OUT] = {
+
+    val javaAsyncFunction = new JavaAsyncFunction[IN, OUT] {
+      override def asyncInvoke(input: IN, collector: JavaAsyncCollector[OUT]): Unit = {
+        asyncFunction.asyncInvoke(input, new JavaAsyncCollectorWrapper(collector))
+      }
+    }
+
+    val outType : TypeInformation[OUT] = implicitly[TypeInformation[OUT]]
+
+    asScalaStream(JavaAsyncDataStream.unorderedWait[IN, OUT](
+      input.javaStream,
+      javaAsyncFunction,
+      timeout,
+      timeUnit,
+      capacity).returns(outType))
+  }
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is only maintained
+    * with respect to watermarks. Stream records which lie between the same two watermarks, can be
+    * re-ordered.
+    *
+    * @param input to apply the async function on
+    * @param asyncFunction to use
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def unorderedWait[IN, OUT: TypeInformation](
+    input: DataStream[IN],
+    asyncFunction: AsyncFunction[IN, OUT],
+    timeout: Long,
+    timeUnit: TimeUnit)
+  : DataStream[OUT] = {
+
+    unorderedWait(input, asyncFunction, timeout, timeUnit, DEFAULT_QUEUE_CAPACITY)
+  }
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is only maintained
+    * with respect to watermarks. Stream records which lie between the same two watermarks, can be
+    * re-ordered.
+    *
+    * @param input to apply the async function on
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @param capacity of the operator which is equivalent to the number of concurrent asynchronous
+    *                 operations
+    * @param asyncFunction to use
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def unorderedWait[IN, OUT: TypeInformation](
+      input: DataStream[IN],
+      timeout: Long,
+      timeUnit: TimeUnit,
+      capacity: Int) (
+      asyncFunction: (IN, AsyncCollector[OUT]) => Unit)
+    : DataStream[OUT] = {
+
+    Preconditions.checkNotNull(asyncFunction)
+
+    val cleanAsyncFunction = input.executionEnvironment.scalaClean(asyncFunction)
+
+    val func = new JavaAsyncFunction[IN, OUT] {
+      override def asyncInvoke(input: IN, collector: JavaAsyncCollector[OUT]): Unit = {
+
+        cleanAsyncFunction(input, new JavaAsyncCollectorWrapper[OUT](collector))
+      }
+    }
+
+    val outType : TypeInformation[OUT] = implicitly[TypeInformation[OUT]]
+
+    asScalaStream(JavaAsyncDataStream.unorderedWait[IN, OUT](
+      input.javaStream,
+      func,
+      timeout,
+      timeUnit,
+      capacity).returns(outType))
+  }
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is only maintained
+    * with respect to watermarks. Stream records which lie between the same two watermarks, can be
+    * re-ordered.
+    *
+    * @param input to apply the async function on
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @param asyncFunction to use
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def unorderedWait[IN, OUT: TypeInformation](
+    input: DataStream[IN],
+    timeout: Long,
+    timeUnit: TimeUnit) (
+    asyncFunction: (IN, AsyncCollector[OUT]) => Unit)
+  : DataStream[OUT] = {
+    unorderedWait(input, timeout, timeUnit, DEFAULT_QUEUE_CAPACITY)(asyncFunction)
+  }
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is the same as the
+    * input order of the elements.
+    *
+    * @param input to apply the async function on
+    * @param asyncFunction to use
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @param capacity of the operator which is equivalent to the number of concurrent asynchronous
+    *                 operations
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def orderedWait[IN, OUT: TypeInformation](
+      input: DataStream[IN],
+      asyncFunction: AsyncFunction[IN, OUT],
+      timeout: Long,
+      timeUnit: TimeUnit,
+      capacity: Int)
+    : DataStream[OUT] = {
+
+    val javaAsyncFunction = new JavaAsyncFunction[IN, OUT] {
+      override def asyncInvoke(input: IN, collector: JavaAsyncCollector[OUT]): Unit = {
+        asyncFunction.asyncInvoke(input, new JavaAsyncCollectorWrapper[OUT](collector))
+      }
+    }
+
+    val outType : TypeInformation[OUT] = implicitly[TypeInformation[OUT]]
+
+    asScalaStream(JavaAsyncDataStream.orderedWait[IN, OUT](
+      input.javaStream,
+      javaAsyncFunction,
+      timeout,
+      timeUnit,
+      capacity).returns(outType))
+  }
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is the same as the
+    * input order of the elements.
+    *
+    * @param input to apply the async function on
+    * @param asyncFunction to use
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def orderedWait[IN, OUT: TypeInformation](
+    input: DataStream[IN],
+    asyncFunction: AsyncFunction[IN, OUT],
+    timeout: Long,
+    timeUnit: TimeUnit)
+  : DataStream[OUT] = {
+
+    orderedWait(input, asyncFunction, timeout, timeUnit, DEFAULT_QUEUE_CAPACITY)
+  }
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is the same as the
+    * input order of the elements.
+    *
+    * @param input to apply the async function on
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @param capacity of the operator which is equivalent to the number of concurrent asynchronous
+    *                 operations
+    * @param asyncFunction to use
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def orderedWait[IN, OUT: TypeInformation](
+    input: DataStream[IN],
+    timeout: Long,
+    timeUnit: TimeUnit,
+    capacity: Int) (
+    asyncFunction: (IN, AsyncCollector[OUT]) => Unit)
+  : DataStream[OUT] = {
+
+    Preconditions.checkNotNull(asyncFunction)
+
+    val cleanAsyncFunction = input.executionEnvironment.scalaClean(asyncFunction)
+
+    val func = new JavaAsyncFunction[IN, OUT] {
+      override def asyncInvoke(input: IN, collector: JavaAsyncCollector[OUT]): Unit = {
+        cleanAsyncFunction(input, new JavaAsyncCollectorWrapper[OUT](collector))
+      }
+    }
+
+    val outType : TypeInformation[OUT] = implicitly[TypeInformation[OUT]]
+
+    asScalaStream(JavaAsyncDataStream.orderedWait[IN, OUT](
+      input.javaStream,
+      func,
+      timeout,
+      timeUnit,
+      capacity).returns(outType))
+  }
+
+  /**
+    * Apply an asynchronous function on the input data stream. The output order is the same as the
+    * input order of the elements.
+    *
+    * @param input to apply the async function on
+    * @param timeout for the asynchronous operation to complete
+    * @param timeUnit of the timeout
+    * @param asyncFunction to use
+    * @tparam IN Type of the input record
+    * @tparam OUT Type of the output record
+    * @return the resulting stream containing the asynchronous results
+    */
+  def orderedWait[IN, OUT: TypeInformation](
+    input: DataStream[IN],
+    timeout: Long,
+    timeUnit: TimeUnit) (
+    asyncFunction: (IN, AsyncCollector[OUT]) => Unit)
+  : DataStream[OUT] = {
+
+    orderedWait(input, timeout, timeUnit, DEFAULT_QUEUE_CAPACITY)(asyncFunction)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncCollector.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncCollector.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncCollector.scala
new file mode 100644
index 0000000..a149c88
--- /dev/null
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncCollector.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.scala.async
+
+import org.apache.flink.annotation.PublicEvolving
+
+/**
+  * The async collector collects data/errors from the user code while processing
+  * asynchronous I/O operations.
+  *
+  * @tparam OUT type of the output element
+  */
+@PublicEvolving
+trait AsyncCollector[OUT] {
+
+  /**
+    * Complete the async collector with a set of result elements.
+    *
+    * Note that it should be called for exactly one time in the user code.
+    * Calling this function for multiple times will cause data lose.
+    *
+    * Put all results in a [[Iterable]] and then issue AsyncCollector.collect(Iterable).
+    *
+    * @param result to complete the async collector with
+    */
+  def collect(result: Iterable[OUT])
+
+  /**
+    * Complete this async collector with an error.
+    *
+    * @param throwable to complete the async collector with
+    */
+  def collect(throwable: Throwable)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncFunction.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncFunction.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncFunction.scala
new file mode 100644
index 0000000..72e3702
--- /dev/null
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncFunction.scala
@@ -0,0 +1,48 @@
+/*
+ * 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.scala.async
+
+import org.apache.flink.annotation.PublicEvolving
+
+/**
+  * A function to trigger async I/O operations.
+  *
+  * For each asyncInvoke an async io operation can be triggered, and once it has been done,
+  * the result can be collected by calling AsyncCollector.collect. For each async operation, its
+  * context is stored in the operator immediately after invoking asyncInvoke, avoiding blocking for
+  * each stream input as long as the internal buffer is not full.
+  *
+  * [[AsyncCollector]] can be passed into callbacks or futures to collect the result data.
+  * An error can also be propagate to the async IO operator by
+  * [[AsyncCollector.collect(Throwable)]].
+  *
+  * @tparam IN The type of the input element
+  * @tparam OUT The type of the output elements
+  */
+@PublicEvolving
+trait AsyncFunction[IN, OUT] {
+
+  /**
+    * Trigger the async operation for each stream input
+    *
+    * @param input element coming from an upstream task
+    * @param collector to collect the result data
+    */
+  def asyncInvoke(input: IN, collector: AsyncCollector[OUT]): Unit
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bfdaa382/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/JavaAsyncCollectorWrapper.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/JavaAsyncCollectorWrapper.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/JavaAsyncCollectorWrapper.scala
new file mode 100644
index 0000000..3c5e95a
--- /dev/null
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/JavaAsyncCollectorWrapper.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.scala.async
+
+import org.apache.flink.annotation.Internal
+import org.apache.flink.streaming.api.functions.async.collector.{AsyncCollector => JavaAsyncCollector}
+
+import scala.collection.JavaConverters._
+
+/**
+  * Internal wrapper class to map a Flink's Java API [[JavaAsyncCollector]] to a Scala
+  * [[AsyncCollector]].
+  *
+  * @param javaAsyncCollector to forward the calls to
+  * @tparam OUT type of the output elements
+  */
+@Internal
+class JavaAsyncCollectorWrapper[OUT](val javaAsyncCollector: JavaAsyncCollector[OUT])
+  extends AsyncCollector[OUT] {
+  override def collect(result: Iterable[OUT]): Unit = {
+    javaAsyncCollector.collect(result.asJavaCollection)
+  }
+
+  override def collect(throwable: Throwable): Unit = {
+    javaAsyncCollector.collect(throwable)
+  }
+}


[2/7] flink git commit: [FLINK-4391] Add asynchronous I/O operations

Posted by tr...@apache.org.
[FLINK-4391] Add asynchronous I/O operations

1. add an example job 2. fix a bug in state serialization in async wait operator; 3. move broadcast barrier after snapshot operator states

update IT case

adjust the whitespace in IT

1. use final for member variable; 2. initialize resouce in open() 3. use ioexception instead of runtimeexception to propagate errors

make sure head operator comes first while doing shapshot for chained operators

[FLINK-4391] 1. adjust the order of snapshot for operators in one chain, so that head operator can do snapshot first. it is for the chained operator with async wait operator, which will keep emitting data in the internal buffer to its children if stream task perform checkpoint from tail to the head, getting incorrect result. 2. support LatencyMarker in async wait operator

[FLINK-4391] use checkpoint lock in async wait operator; remove emitter thread

[FLINK-4391] use checkpoint lock in async wait operator; remove emitter thread

[FLINK-4391] 1. Re-add emitter thread. Without this thread, if there is no input coming, and we just use main thread to emit result, the finished async collectors may have to wait uncertained period of time to be emitted. The emitter thread can help output them as soos as possible; 2. In UNORDERED mode, only emits results prior to the oldest Watermark in the buffer; 3. Use the latest OperatorStateStore to keep partitionable operator state.

[FLINK-4391] remove change to StreamTask.java

[FLINK-4391] Optimize inner data structure for AsyncWaitOperator, add extra test cases.

[FLINK-4391] Fix UT failure

[FLINK-4391] Fix format problem

[FLINK-4391] Add a RuntimeContext wrapper for RichAsyncFunction to disable getting state from RuntimeContext.

This closes #2629.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/f5283076
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/f5283076
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/f5283076

Branch: refs/heads/master
Commit: f52830763d8f95a955c10265e2c3543a5890e719
Parents: 4a27d21
Author: yushi.wxg <yu...@taobao.com>
Authored: Wed Oct 12 14:31:43 2016 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Dec 20 05:04:07 2016 +0100

----------------------------------------------------------------------
 flink-examples/flink-examples-streaming/pom.xml |  22 +
 .../examples/async/AsyncIOExample.java          | 277 ++++++++
 .../api/datastream/AsyncDataStream.java         | 135 ++++
 .../api/functions/async/AsyncFunction.java      |  88 +++
 .../api/functions/async/RichAsyncFunction.java  | 245 +++++++
 .../async/buffer/AbstractBufferEntry.java       |  78 +++
 .../async/buffer/AsyncCollectorBuffer.java      | 633 ++++++++++++++++++
 .../async/buffer/LatencyMarkerEntry.java        |  36 +
 .../async/buffer/StreamElementEntry.java        |  82 +++
 .../async/buffer/StreamRecordEntry.java         |  75 +++
 .../functions/async/buffer/WatermarkEntry.java  |  36 +
 .../async/collector/AsyncCollector.java         |  53 ++
 .../api/operators/AbstractStreamOperator.java   |   6 +-
 .../api/operators/async/AsyncWaitOperator.java  | 211 ++++++
 .../streaming/runtime/tasks/OperatorChain.java  |  25 +-
 .../streaming/runtime/tasks/StreamTask.java     |   3 +-
 .../functions/async/RichAsyncFunctionTest.java  | 164 +++++
 .../async/AsyncCollectorBufferTest.java         | 656 +++++++++++++++++++
 .../operators/async/AsyncWaitOperatorTest.java  | 629 ++++++++++++++++++
 .../operators/StreamOperatorChainingTest.java   |   2 +
 .../runtime/tasks/StreamTaskTestHarness.java    |   4 +
 .../util/OneInputStreamOperatorTestHarness.java |  10 +
 .../streaming/api/StreamingOperatorsITCase.java | 208 ++++--
 23 files changed, 3615 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-examples/flink-examples-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml
index c418ce7..f2891c9 100644
--- a/flink-examples/flink-examples-streaming/pom.xml
+++ b/flink-examples/flink-examples-streaming/pom.xml
@@ -221,6 +221,28 @@ under the License.
 						</configuration>
 					</execution>
 
+					<!-- Async I/O -->
+					<execution>
+						<id>AsyncIO</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>AsyncIO</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.async.AsyncIOExample</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/async/*.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
 					<!-- WordCountPOJO -->
 					<execution>
 						<id>WordCountPOJO</id>

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
new file mode 100644
index 0000000..96c7658
--- /dev/null
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.examples.async;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.functions.async.RichAsyncFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Example to illustrates how to use {@link org.apache.flink.streaming.api.functions.async.AsyncFunction}
+ */
+public class AsyncIOExample {
+
+	/**
+	 * A checkpointed source.
+	 */
+	private static class SimpleSource implements SourceFunction<Integer>, ListCheckpointed<Integer> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean isRunning = true;
+		private int counter = 0;
+		private int start = 0;
+
+		@Override
+		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(start);
+		}
+
+		@Override
+		public void restoreState(List<Integer> state) throws Exception {
+			for (Integer i : state)
+				this.start = i;
+		}
+
+		public SimpleSource(int maxNum) {
+			this.counter = maxNum;
+		}
+
+		@Override
+		public void run(SourceContext<Integer> ctx) throws Exception {
+			while ((start < counter || counter == -1) && isRunning) {
+				synchronized (ctx.getCheckpointLock()) {
+					ctx.collect(start);
+					++start;
+
+					// loop back to 0
+					if (start == Integer.MAX_VALUE) {
+						start = 0;
+					}
+				}
+				Thread.sleep(10);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+	}
+
+
+	/**
+	 * An sample of {@link AsyncFunction} using a thread pool and executing working threads
+	 * to simulate multiple async operations.
+	 * <p>
+	 * For the real use case in production environment, the thread pool may stay in the
+	 * async client.
+	 */
+	private static class SampleAsyncFunction extends RichAsyncFunction<Integer, String> {
+		transient static ExecutorService executorService;
+		transient static Random random;
+
+		private int counter;
+
+		/**
+		 * The result of multiplying sleepFactor with a random float is used to pause
+		 * the working thread in the thread pool, simulating a time consuming async operation.
+		 */
+		final long sleepFactor;
+
+		/**
+		 * The ratio to generate an exception to simulate an async error. For example, the error
+		 * may be a TimeoutException while visiting HBase.
+		 */
+		final float failRatio;
+
+		final long shutdownWaitTS;
+
+		public SampleAsyncFunction(long sleepFactor, float failRatio, long shutdownWaitTS) {
+			this.sleepFactor = sleepFactor;
+			this.failRatio = failRatio;
+			this.shutdownWaitTS = shutdownWaitTS;
+		}
+
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			synchronized (SampleAsyncFunction.class) {
+				if (counter == 0) {
+					executorService = Executors.newFixedThreadPool(30);
+
+					random = new Random();
+				}
+
+				++counter;
+			}
+		}
+
+		@Override
+		public void close() throws Exception {
+			super.close();
+
+			synchronized (SampleAsyncFunction.class) {
+				--counter;
+
+				if (counter == 0) {
+					executorService.shutdown();
+
+					try {
+						executorService.awaitTermination(shutdownWaitTS, TimeUnit.MILLISECONDS);
+					} catch (InterruptedException e) {
+						executorService.shutdownNow();
+					}
+				}
+			}
+		}
+
+		@Override
+		public void asyncInvoke(final Integer input, final AsyncCollector<String> collector) throws Exception {
+			this.executorService.submit(new Runnable() {
+				@Override
+				public void run() {
+					// wait for while to simulate async operation here
+					int sleep = (int) (random.nextFloat() * sleepFactor);
+					try {
+						Thread.sleep(sleep);
+						List<String> ret = Collections.singletonList("key-" + (input % 10));
+						if (random.nextFloat() < failRatio) {
+							collector.collect(new Exception("wahahahaha..."));
+						} else {
+							collector.collect(ret);
+						}
+					} catch (InterruptedException e) {
+						collector.collect(new ArrayList<String>(0));
+					}
+				}
+			});
+		}
+	}
+
+	private static void printUsage() {
+		System.out.println("To customize example, use: AsyncIOExample [--fsStatePath <path to fs state>] " +
+				"[--checkpointMode <exactly_once or at_least_once>] " +
+				"[--maxCount <max number of input from source, -1 for infinite input>] " +
+				"[--sleepFactor <interval to sleep for each stream element>] [--failRatio <possibility to throw exception>] " +
+				"[--waitMode <ordered or unordered>] [--waitOperatorParallelism <parallelism for async wait operator>] " +
+				"[--eventType <EventTime or IngestionTime>] [--shutdownWaitTS <milli sec to wait for thread pool>]");
+	}
+
+	public static void main(String[] args) throws Exception {
+
+		// obtain execution environment
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		printUsage();
+
+		// parse parameters
+		final ParameterTool params = ParameterTool.fromArgs(args);
+
+		// check the configuration for the job
+		final String statePath = params.getRequired("fsStatePath");
+		final String cpMode = params.get("checkpointMode", "exactly_once");
+		final int maxCount = params.getInt("maxCount", 100000);
+		final int sleepFactor = params.getInt("sleepFactor", 100);
+		final float failRatio = params.getFloat("failRatio", 0.001f);
+		final String mode = params.get("waitMode", "ordered");
+		final int taskNum =  params.getInt("waitOperatorParallelism", 1);
+		final String timeType = params.get("eventType", "EventTime");
+		final int shutdownWaitTS = params.getInt("shutdownWaitTS", 20000);
+
+		System.out.println("Job configuration\n"
+			+"\tFS state path="+statePath+"\n"
+			+"\tCheckpoint mode="+cpMode+"\n"
+			+"\tMax count of input from source="+maxCount+"\n"
+			+"\tSleep factor="+sleepFactor+"\n"
+			+"\tFail ratio="+failRatio+"\n"
+			+"\tWaiting mode="+mode+"\n"
+			+"\tParallelism for async wait operator="+taskNum+"\n"
+			+"\tEvent type="+timeType+"\n"
+			+"\tShutdown wait timestamp="+shutdownWaitTS);
+
+		// setup state and checkpoint mode
+		env.setStateBackend(new FsStateBackend(statePath));
+		if (cpMode.equals("exactly_once")) {
+			env.enableCheckpointing(1000, CheckpointingMode.EXACTLY_ONCE);
+		}
+		else {
+			env.enableCheckpointing(1000, CheckpointingMode.AT_LEAST_ONCE);
+		}
+
+		// enable watermark or not
+		if (timeType.equals("EventTime")) {
+			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		}
+		else if (timeType.equals("IngestionTime")) {
+			env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
+		}
+
+		// create input stream of an single integer
+		DataStream<Integer> inputStream = env.addSource(new SimpleSource(maxCount));
+
+		// create async function, which will *wait* for a while to simulate the process of async i/o
+		AsyncFunction<Integer, String> function =
+				new SampleAsyncFunction(sleepFactor, failRatio, shutdownWaitTS);
+
+		// add async operator to streaming job
+		DataStream<String> result;
+		if (mode.equals("ordered")) {
+			result = AsyncDataStream.orderedWait(inputStream, function, 20).setParallelism(taskNum);
+		}
+		else {
+			result = AsyncDataStream.unorderedWait(inputStream, function, 20).setParallelism(taskNum);
+		}
+
+		// add a reduce to get the sum of each keys.
+		result.flatMap(new FlatMapFunction<String, Tuple2<String, Integer>>() {
+			@Override
+			public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception {
+				out.collect(new Tuple2<>(value, 1));
+			}
+		}).keyBy(0).sum(1).print();
+
+		// execute the program
+		env.execute("Async I/O Example");
+	}
+}
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java
new file mode 100644
index 0000000..4fefde0
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.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.datastream;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
+
+/**
+ * A helper class to apply {@link AsyncFunction} to a data stream.
+ * <p>
+ * <pre>{@code
+ * DataStream<String> input = ...
+ * AsyncFunction<String, Tuple<String, String>> asyncFunc = ...
+ *
+ * AsyncDataStream.orderedWait(input, asyncFunc, 100);
+ * }
+ * </pre>
+ */
+
+@PublicEvolving
+public class AsyncDataStream {
+	public enum OutputMode { ORDERED, UNORDERED }
+
+	private static final int DEFAULT_BUFFER_SIZE = 100;
+
+	/**
+	 * Add an AsyncWaitOperator.
+	 *
+	 * @param in The {@link DataStream} where the {@link AsyncWaitOperator} will be added.
+	 * @param func {@link AsyncFunction} wrapped inside {@link AsyncWaitOperator}.
+	 * @param bufSize The max number of inputs the {@link AsyncWaitOperator} can hold inside.
+	 * @param mode Processing mode for {@link AsyncWaitOperator}.
+	 * @param <IN> Input type.
+	 * @param <OUT> Output type.
+	 * @return A new {@link SingleOutputStreamOperator}
+	 */
+	private static <IN, OUT> SingleOutputStreamOperator<OUT> addOperator(
+			DataStream<IN> in,
+			AsyncFunction<IN, OUT> func,
+			int bufSize,
+			OutputMode mode) {
+
+		TypeInformation<OUT> outTypeInfo =
+			TypeExtractor.getUnaryOperatorReturnType(func, AsyncFunction.class, false,
+				true, in.getType(), Utils.getCallLocationName(), true);
+
+		// create transform
+		AsyncWaitOperator<IN, OUT> operator =
+				new AsyncWaitOperator<>(in.getExecutionEnvironment().clean(func), bufSize, mode);
+
+		return in.transform("async wait operator", outTypeInfo, operator);
+	}
+
+	/**
+	 * Add an AsyncWaitOperator. The order of output stream records may be reordered.
+	 *
+	 * @param in Input {@link DataStream}
+	 * @param func {@link AsyncFunction}
+	 * @param bufSize The max number of async i/o operation that can be triggered
+	 * @param <IN> Type of input record
+	 * @param <OUT> Type of output record
+	 * @return A new {@link SingleOutputStreamOperator}.
+	 */
+	public static <IN, OUT> SingleOutputStreamOperator<OUT> unorderedWait(
+			DataStream<IN> in,
+			AsyncFunction<IN, OUT> func,
+			int bufSize) {
+		return addOperator(in, func, bufSize, OutputMode.UNORDERED);
+	}
+
+	/**
+	 * Add an AsyncWaitOperator. The order of output stream records may be reordered.
+	 * @param in Input {@link DataStream}
+	 * @param func {@link AsyncFunction}
+	 * @param <IN> Type of input record
+	 * @param <OUT> Type of output record
+	 * @return A new {@link SingleOutputStreamOperator}.
+	 */
+	public static <IN, OUT> SingleOutputStreamOperator<OUT> unorderedWait(
+			DataStream<IN> in,
+			AsyncFunction<IN, OUT> func) {
+		return addOperator(in, func, DEFAULT_BUFFER_SIZE, OutputMode.UNORDERED);
+	}
+
+	/**
+	 * Add an AsyncWaitOperator. The order to process input records is guaranteed to be the same as input ones.
+	 *
+	 * @param in Input {@link DataStream}
+	 * @param func {@link AsyncFunction}
+	 * @param bufSize The max number of async i/o operation that can be triggered
+	 * @param <IN> Type of input record
+	 * @param <OUT> Type of output record
+	 * @return A new {@link SingleOutputStreamOperator}.
+	 */
+	public static <IN, OUT> SingleOutputStreamOperator<OUT> orderedWait(
+			DataStream<IN> in,
+			AsyncFunction<IN, OUT> func,
+			int bufSize) {
+		return addOperator(in, func, bufSize, OutputMode.ORDERED);
+	}
+
+	/**
+	 * Add an AsyncWaitOperator. The order to process input records is guaranteed to be the same as input ones.
+	 *
+	 * @param in Input {@link DataStream}
+	 * @param func {@link AsyncFunction}
+	 * @param <IN> Type of input record
+	 * @param <OUT> Type of output record
+	 * @return A new {@link SingleOutputStreamOperator}.
+	 */
+	public static <IN, OUT> SingleOutputStreamOperator<OUT> orderedWait(
+			DataStream<IN> in,
+			AsyncFunction<IN, OUT> func) {
+		return addOperator(in, func, DEFAULT_BUFFER_SIZE, OutputMode.ORDERED);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
new file mode 100644
index 0000000..b5b7d6f
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
@@ -0,0 +1,88 @@
+/*
+ * 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.async;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+
+import java.io.Serializable;
+
+/**
+ * A function to trigger Async I/O operation.
+ * <p>
+ * For each #asyncInvoke, an async io operation can be triggered, and once it has been done,
+ * the result can be collected by calling {@link AsyncCollector#collect}. For each async
+ * operations, their contexts are buffered in the operator immediately after invoking
+ * #asyncInvoke, leading to no blocking for each stream input as long as internal buffer is not full.
+ * <p>
+ * {@link AsyncCollector} can be passed into callbacks or futures provided by async client to
+ * fetch result data. Any error can also be propagate to the operator by {@link AsyncCollector#collect(Throwable)}.
+ *
+ * <p>
+ * Typical usage for callback:
+ * <pre>{@code
+ * public class HBaseAsyncFunc implements AsyncFunction<String, String> {
+ *   @Override
+ *   public void asyncInvoke(String row, AsyncCollector<String> collector) throws Exception {
+ *     HBaseCallback cb = new HBaseCallback(collector);
+ *     Get get = new Get(Bytes.toBytes(row));
+ *     hbase.asyncGet(get, cb);
+ *   }
+ * }
+ * }
+ * </pre>
+ *
+ * <p>
+ * Typical usage for {@link com.google.common.util.concurrent.ListenableFuture}
+ * <pre>{@code
+ * public class HBaseAsyncFunc implements AsyncFunction<String, String> {
+ *   @Override
+ *   public void asyncInvoke(String row, final AsyncCollector<String> collector) throws Exception {
+ *     Get get = new Get(Bytes.toBytes(row));
+ *     ListenableFuture<Result> future = hbase.asyncGet(get);
+ *     Futures.addCallback(future, new FutureCallback<Result>() {
+ *       public void onSuccess(Result result) {
+ *         List<String> ret = process(result);
+ *         collector.collect(ret);
+ *       }
+ *       public void onFailure(Throwable thrown) {
+ *         collector.collect(thrown);
+ *       }
+ *     });
+ *   }
+ * }
+ * }
+ * </pre>
+ *
+ * @param <IN> The type of the input elements.
+ * @param <OUT> The type of the returned elements.
+ */
+
+@PublicEvolving
+public interface AsyncFunction<IN, OUT> extends Function, Serializable {
+	/**
+	 * Trigger async operation for each stream input.
+	 *
+	 * @param input Stream Input
+	 * @param collector AsyncCollector
+	 * @exception Exception will make task fail and trigger fail-over process.
+	 */
+	void asyncInvoke(IN input, AsyncCollector<OUT> collector) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java
new file mode 100644
index 0000000..f6d3d31
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java
@@ -0,0 +1,245 @@
+/*
+ * 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.async;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.DoubleCounter;
+import org.apache.flink.api.common.accumulators.Histogram;
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.cache.DistributedCache;
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.types.Value;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Rich variant of the {@link AsyncFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ *
+ * <p>
+ * State related apis in {@link RuntimeContext} are not supported yet because the key may get changed
+ * while accessing states in the working thread.
+ * <p>
+ * {@link IterationRuntimeContext#getIterationAggregator(String)} is not supported since the aggregator
+ * may be modified by multiple threads.
+ *
+ * @param <IN> The type of the input elements.
+ * @param <OUT> The type of the returned elements.
+ */
+
+@PublicEvolving
+public abstract class RichAsyncFunction<IN, OUT> extends AbstractRichFunction
+	implements AsyncFunction<IN, OUT> {
+
+	private transient RuntimeContext runtimeContext;
+
+	@Override
+	public void setRuntimeContext(RuntimeContext t) {
+		super.setRuntimeContext(t);
+
+		if (t != null) {
+			runtimeContext = new RichAsyncFunctionRuntimeContext(t);
+		}
+	}
+
+	@Override
+	public abstract void asyncInvoke(IN input, AsyncCollector<OUT> collector) throws Exception;
+
+	@Override
+	public RuntimeContext getRuntimeContext() {
+		if (this.runtimeContext != null) {
+			return runtimeContext;
+		} else {
+			throw new IllegalStateException("The runtime context has not been initialized.");
+		}
+	}
+
+	@Override
+	public IterationRuntimeContext getIterationRuntimeContext() {
+		if (this.runtimeContext != null) {
+			return (IterationRuntimeContext) runtimeContext;
+		} else {
+			throw new IllegalStateException("The runtime context has not been initialized.");
+		}
+	}
+
+	/**
+	 * A wrapper class to delegate {@link RuntimeContext}. State related apis are disabled.
+	 */
+	private class RichAsyncFunctionRuntimeContext implements IterationRuntimeContext {
+		private RuntimeContext runtimeContext;
+
+		public RichAsyncFunctionRuntimeContext(RuntimeContext context) {
+			runtimeContext = context;
+		}
+
+		private IterationRuntimeContext getIterationRuntineContext() {
+			if (this.runtimeContext instanceof IterationRuntimeContext) {
+				return (IterationRuntimeContext) this.runtimeContext;
+			} else {
+				throw new IllegalStateException("This stub is not part of an iteration step function.");
+			}
+		}
+
+		@Override
+		public int getSuperstepNumber() {
+			return getIterationRuntineContext().getSuperstepNumber();
+		}
+
+		@Override
+		public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+			throw new UnsupportedOperationException("Get iteration aggregator is not supported in rich async function");
+		}
+
+		@Override
+		public <T extends Value> T getPreviousIterationAggregate(String name) {
+			return getIterationRuntineContext().getPreviousIterationAggregate(name);
+		}
+
+		@Override
+		public String getTaskName() {
+			return runtimeContext.getTaskName();
+		}
+
+		@Override
+		public MetricGroup getMetricGroup() {
+			return runtimeContext.getMetricGroup();
+		}
+
+		@Override
+		public int getNumberOfParallelSubtasks() {
+			return runtimeContext.getNumberOfParallelSubtasks();
+		}
+
+		@Override
+		public int getIndexOfThisSubtask() {
+			return runtimeContext.getIndexOfThisSubtask();
+		}
+
+		@Override
+		public int getAttemptNumber() {
+			return runtimeContext.getAttemptNumber();
+		}
+
+		@Override
+		public String getTaskNameWithSubtasks() {
+			return runtimeContext.getTaskNameWithSubtasks();
+		}
+
+		@Override
+		public ExecutionConfig getExecutionConfig() {
+			return runtimeContext.getExecutionConfig();
+		}
+
+		@Override
+		public ClassLoader getUserCodeClassLoader() {
+			return runtimeContext.getUserCodeClassLoader();
+		}
+
+		@Override
+		public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {
+			runtimeContext.addAccumulator(name, accumulator);
+		}
+
+		@Override
+		public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
+			return runtimeContext.getAccumulator(name);
+		}
+
+		@Override
+		public Map<String, Accumulator<?, ?>> getAllAccumulators() {
+			return runtimeContext.getAllAccumulators();
+		}
+
+		@Override
+		public IntCounter getIntCounter(String name) {
+			return runtimeContext.getIntCounter(name);
+		}
+
+		@Override
+		public LongCounter getLongCounter(String name) {
+			return runtimeContext.getLongCounter(name);
+		}
+
+		@Override
+		public DoubleCounter getDoubleCounter(String name) {
+			return runtimeContext.getDoubleCounter(name);
+		}
+
+		@Override
+		public Histogram getHistogram(String name) {
+			return runtimeContext.getHistogram(name);
+		}
+
+		@Override
+		public boolean hasBroadcastVariable(String name) {
+			return runtimeContext.hasBroadcastVariable(name);
+		}
+
+		@Override
+		public <RT> List<RT> getBroadcastVariable(String name) {
+			return runtimeContext.getBroadcastVariable(name);
+		}
+
+		@Override
+		public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
+			return runtimeContext.getBroadcastVariableWithInitializer(name, initializer);
+		}
+
+		@Override
+		public DistributedCache getDistributedCache() {
+			return runtimeContext.getDistributedCache();
+		}
+
+		@Override
+		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("State is not supported in rich async function");
+		}
+
+		@Override
+		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("State is not supported in rich async function");
+		}
+
+		@Override
+		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("State is not supported in rich async function");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java
new file mode 100644
index 0000000..29643fd
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java
@@ -0,0 +1,78 @@
+/*
+ * 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.async.buffer;
+
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Abstract implementation for {@link StreamElementEntry}
+ *
+ * @param <OUT> Output type.
+ */
+public abstract class AbstractBufferEntry<OUT> implements StreamElementEntry<OUT> {
+	private final StreamElement streamElement;
+
+	protected AbstractBufferEntry(StreamElement element) {
+		this.streamElement = Preconditions.checkNotNull(element, "Reference to StreamElement should not be null");
+	}
+
+	@Override
+	public List<OUT> getResult() throws IOException {
+		throw new UnsupportedOperationException("It is only available for StreamRecordEntry");
+	}
+
+	@Override
+	public void markDone() {
+		throw new UnsupportedOperationException("It is only available for StreamRecordEntry");
+	}
+
+	@Override
+	public boolean isDone() {
+		throw new UnsupportedOperationException("It must be overriden by the concrete entry");
+	}
+
+	@Override
+	public boolean isStreamRecord() {
+		return streamElement.isRecord();
+	}
+
+	@Override
+	public boolean isWatermark() {
+		return streamElement.isWatermark();
+	}
+
+	@Override
+	public boolean isLatencyMarker() {
+		return streamElement.isLatencyMarker();
+	}
+
+	@Override
+	public StreamElement getStreamElement() {
+		return streamElement;
+	}
+
+	@Override
+	public String toString() {
+		return "StreamElementEntry for @" + streamElement;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java
new file mode 100644
index 0000000..ee176d9
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java
@@ -0,0 +1,633 @@
+/*
+ * 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.async.buffer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * AsyncCollectorBuffer will hold all {@link AsyncCollector} in its internal buffer,
+ * and emit results from {@link AsyncCollector} to the next operators following it by
+ * calling {@link Output#collect(Object)}
+ */
+@Internal
+public class AsyncCollectorBuffer<IN, OUT> {
+
+	/**
+	 * The logger.
+	 */
+	private static final Logger LOG = LoggerFactory.getLogger(AsyncCollectorBuffer.class);
+
+	/**
+	 * Max number of {@link AsyncCollector} in the buffer.
+	 */
+	private final int bufferSize;
+
+	private final AsyncDataStream.OutputMode mode;
+
+	private final AsyncWaitOperator<IN, OUT> operator;
+
+	/**
+	 * Keep all {@link StreamElementEntry}
+	 */
+	private final Set<StreamElementEntry<OUT>> queue = new LinkedHashSet<>();
+
+	/**
+	 * Keep all {@link StreamElementEntry} to their corresponding {@link Watermark} or {@link LatencyMarker}
+	 * If the inputs are: SR1, SR2, WM1, SR3, SR4. Then SR1 and SR2 belong to WM1, and
+	 * SR3 and SR4 will be kept in {@link #lonelyEntries}
+	 */
+	private final Map<StreamElementEntry<OUT>, StreamElement> entriesToMarkers = new HashMap<>();
+
+	private final List<StreamElementEntry<OUT>> lonelyEntries = new LinkedList<>();
+
+	/**
+	 * Keep finished AsyncCollector belonging to the oldest Watermark or LatencyMarker in UNORDERED mode.
+	 */
+	private final Map<StreamElement, Set<StreamElementEntry<OUT>>> markerToFinishedEntries = new LinkedHashMap<>();
+	private Set<StreamElementEntry<OUT>>lonelyFinishedEntries = new HashSet<>();
+
+	/**
+	 * For the AsyncWaitOperator chained with StreamSource, the checkpoint thread may get the
+	 * {@link org.apache.flink.streaming.runtime.tasks.StreamTask#lock} while {@link AsyncCollectorBuffer#queue}
+	 * is full since main thread waits on this lock. The StreamElement in
+	 * {@link AsyncWaitOperator#processElement(StreamRecord)} should be treated as a part of all StreamElements
+	 * in its queue. It will be kept in the operator state while snapshotting.
+	 */
+	private StreamElement extraStreamElement;
+
+	/**
+	 * {@link TimestampedCollector} and {@link Output} to collect results and watermarks.
+	 */
+	private final Output<StreamRecord<OUT>> output;
+	private final TimestampedCollector<OUT> timestampedCollector;
+
+	/**
+	 * Checkpoint lock from {@link org.apache.flink.streaming.runtime.tasks.StreamTask#lock}
+	 */
+	private final Object lock;
+
+	private final Emitter emitter;
+	private final Thread emitThread;
+
+	/**
+	 * Exception from async operation or internal error
+	 */
+	private Exception error;
+
+	/**
+	 * Flag telling Emitter thread to work or not.
+	 */
+	private volatile boolean workwork = false;
+
+	public AsyncCollectorBuffer(
+			int bufferSize,
+			AsyncDataStream.OutputMode mode,
+			Output<StreamRecord<OUT>> output,
+			TimestampedCollector<OUT> collector,
+			Object lock,
+			AsyncWaitOperator operator) {
+		Preconditions.checkArgument(bufferSize > 0, "Future buffer size should be greater than 0.");
+
+		this.bufferSize = bufferSize;
+
+		this.mode = Preconditions.checkNotNull(mode, "Processing mode should not be NULL.");
+		this.output = Preconditions.checkNotNull(output, "Output should not be NULL.");
+		this.timestampedCollector = Preconditions.checkNotNull(collector, "TimestampedCollector should not be NULL.");
+		this.operator = Preconditions.checkNotNull(operator, "Reference to AsyncWaitOperator should not be NULL.");
+		this.lock = Preconditions.checkNotNull(lock, "Checkpoint lock should not be NULL.");
+
+		this.emitter = new Emitter();
+		this.emitThread = new Thread(emitter);
+		this.emitThread.setDaemon(true);
+	}
+
+	/**
+	 * Add an {@link StreamRecord} into the buffer. A new {@link AsyncCollector} will be created and returned
+	 * corresponding to the input StreamRecord.
+	 * <p>
+	 * If buffer is full, caller will wait until a new space is available.
+	 *
+	 * @param record StreamRecord
+	 * @return An AsyncCollector
+	 * @throws Exception Exception from AsyncCollector.
+	 */
+	public AsyncCollector<OUT> addStreamRecord(StreamRecord<IN> record) throws Exception {
+		assert(Thread.holdsLock(lock));
+
+		while (queue.size() >= bufferSize) {
+			// hold the input StreamRecord until it is placed in the buffer
+			extraStreamElement = record;
+
+			lock.wait();
+		}
+
+		if (error != null) {
+			throw error;
+		}
+
+		StreamElementEntry<OUT> entry = new StreamRecordEntry<>(record, this);
+
+		queue.add(entry);
+
+		if (mode == AsyncDataStream.OutputMode.UNORDERED) {
+			lonelyEntries.add(entry);
+		}
+
+		extraStreamElement = null;
+
+		return (AsyncCollector<OUT>)entry;
+	}
+
+	/**
+	 * Add a {@link Watermark} into buffer.
+	 * <p>
+	 * If queue is full, caller will wait here.
+	 *
+	 * @param watermark Watermark
+	 * @throws Exception Exception from AsyncCollector.
+	 */
+	public void addWatermark(Watermark watermark) throws Exception {
+		processMark(new WatermarkEntry<OUT>(watermark));
+	}
+
+	/**
+	 * Add a {@link LatencyMarker} into buffer.
+	 * <p>
+	 * If queue is full, caller will wait here.
+	 *
+	 * @param latencyMarker LatencyMarker
+	 * @throws Exception Exception from AsyncCollector.
+	 */
+	public void addLatencyMarker(LatencyMarker latencyMarker) throws Exception {
+		processMark(new LatencyMarkerEntry<OUT>(latencyMarker));
+	}
+
+	/**
+	 * Notify the emitter thread and main thread that an AsyncCollector has completed.
+	 *
+	 * @param entry Completed AsyncCollector
+	 */
+	public void markCollectorCompleted(StreamElementEntry<OUT> entry) {
+		synchronized (lock) {
+			entry.markDone();
+
+			if (mode == AsyncDataStream.OutputMode.UNORDERED) {
+				StreamElement marker = entriesToMarkers.get(entry);
+
+				if (marker != null) {
+					markerToFinishedEntries.get(marker).add(entry);
+				}
+				else {
+					lonelyFinishedEntries.add(entry);
+				}
+			}
+
+			// if workwork is true, it is not necessary to check it again
+			if (!workwork && shouldNotifyEmitterThread(entry)) {
+				workwork = true;
+
+				lock.notifyAll();
+			}
+		}
+	}
+
+	/**
+	 * Caller will wait here if buffer is not empty, meaning that not all async i/o tasks have returned yet.
+	 *
+	 * @throws Exception IOException from AsyncCollector.
+	 */
+	public void waitEmpty() throws Exception {
+		assert(Thread.holdsLock(lock));
+
+		while (queue.size() != 0) {
+			if (error != null) {
+				throw error;
+			}
+
+			lock.wait();
+		}
+	}
+
+	public void startEmitterThread() {
+		emitThread.start();
+	}
+
+	public void stopEmitterThread() {
+		emitter.stop();
+
+		emitThread.interrupt();
+
+		while (emitThread.isAlive()) {
+			// temporarily release the lock first, since caller of this method may also hold the lock.
+			if (Thread.holdsLock(lock)) {
+				try {
+					lock.wait(1000);
+				}
+				catch (InterruptedException e) {
+					// do nothing
+				}
+			}
+
+			try {
+				emitThread.join(10000);
+			} catch (InterruptedException e) {
+				// do nothing
+			}
+
+			// get the stack trace
+			StringBuilder sb = new StringBuilder();
+			StackTraceElement[] stack = emitThread.getStackTrace();
+
+			for (StackTraceElement e : stack) {
+				sb.append(e).append('\n');
+			}
+
+			LOG.warn("Emitter thread blocks due to {}", sb.toString());
+
+			emitThread.interrupt();
+		}
+	}
+
+	/**
+	 * Get all StreamElements in the AsyncCollector queue.
+	 * <p>
+	 * Emitter Thread can not output records and will wait for a while due to checkpoiting procedure
+	 * holding the checkpoint lock.
+	 *
+	 * @return An {@link Iterator} to the StreamElements in the buffer, including the extra one.
+	 */
+	public Iterator<StreamElement> getStreamElementsInBuffer() {
+		final Iterator<StreamElementEntry<OUT>> iterator = queue.iterator();
+		final StreamElement extra = extraStreamElement;
+
+		return new Iterator<StreamElement>() {
+			boolean shouldSendExtraElement = (extra != null);
+
+			@Override
+			public boolean hasNext() {
+				return iterator.hasNext() || shouldSendExtraElement;
+			}
+
+			@Override
+			public StreamElement next() {
+				if (!hasNext()) {
+					throw new NoSuchElementException();
+				}
+
+				if (iterator.hasNext()) {
+					return iterator.next().getStreamElement();
+				}
+				else {
+					shouldSendExtraElement = false;
+
+					return extra;
+				}
+			}
+
+			@Override
+			public void remove() {
+				throw new UnsupportedOperationException("remove");
+			}
+		};
+	}
+
+	private void processMark(StreamElementEntry<OUT> entry) throws Exception {
+		assert(Thread.holdsLock(lock));
+
+		StreamElement mark = entry.getStreamElement();
+
+		while (queue.size() >= bufferSize) {
+			// hold the input StreamRecord until it is placed in the buffer
+			extraStreamElement = mark;
+
+			lock.wait();
+		}
+
+		if (error != null) {
+			throw error;
+		}
+
+		queue.add(entry);
+
+		if (mode == AsyncDataStream.OutputMode.UNORDERED) {
+			// update AsyncCollector to Watermark / LatencyMarker map
+			for (StreamElementEntry<OUT> e : lonelyEntries) {
+				entriesToMarkers.put(e, mark);
+			}
+
+			lonelyEntries.clear();
+
+			// update Watermark / LatencyMarker to finished AsyncCollector map
+			markerToFinishedEntries.put(mark, lonelyFinishedEntries);
+
+			lonelyFinishedEntries = new HashSet<>();
+		}
+
+		extraStreamElement = null;
+
+		// notify Emitter thread if the head of buffer is Watermark or LatencyMarker
+		// this is for the case when LatencyMarkers keep coming but there is no StreamRecords.
+		StreamElementEntry<OUT> element = queue.iterator().next();
+
+		if (element.isLatencyMarker() || element.isWatermark()) {
+			workwork = true;
+
+			lock.notifyAll();
+		}
+	}
+
+	private boolean shouldNotifyEmitterThread(StreamElementEntry<OUT> entry) {
+
+		switch (mode) {
+
+			case ORDERED:
+				Iterator<StreamElementEntry<OUT>> queueIterator = queue.iterator();
+
+				// get to work as long as the first AsyncCollect is done.
+				return queueIterator.hasNext() && (queueIterator.next().isDone());
+
+			case UNORDERED:
+				Iterator<Map.Entry<StreamElement, Set<StreamElementEntry<OUT>>>> iteratorMarker =
+						markerToFinishedEntries.entrySet().iterator();
+
+				// get to work only the finished AsyncCollector belongs to the oldest Watermark or LatencyMarker
+				// or no Watermark / LatencyMarker is in the buffer yet.
+				return iteratorMarker.hasNext() ? iteratorMarker.next().getValue().contains(entry)
+						: lonelyFinishedEntries.contains(entry);
+
+			default:
+				// this case should never happen
+				return false;
+		}
+	}
+
+	@VisibleForTesting
+	public Set<StreamElementEntry<OUT>> getQueue() {
+		return queue;
+	}
+
+	@VisibleForTesting
+	public void setExtraStreamElement(StreamElement element) {
+		extraStreamElement = element;
+	}
+
+	/**
+	 * A working thread to output results from {@link AsyncCollector} to the next operator.
+	 */
+	private class Emitter implements Runnable {
+		private volatile boolean running = true;
+
+		private void output(StreamElementEntry<OUT> entry) throws Exception {
+
+			StreamElement element = entry.getStreamElement();
+
+			if (element == null) {
+				throw new Exception("StreamElement in the buffer entry should not be null");
+			}
+
+			if (entry.isStreamRecord()) {
+				List<OUT> result = entry.getResult();
+
+				if (result == null) {
+					throw new Exception("Result for stream record " + element + " is null");
+				}
+
+				// update the timestamp for the collector
+				timestampedCollector.setTimestamp(element.asRecord());
+
+				for (OUT val : result) {
+					timestampedCollector.collect(val);
+				}
+			}
+			else if (entry.isWatermark()) {
+				output.emitWatermark(element.asWatermark());
+			}
+			else if (entry.isLatencyMarker()) {
+				operator.sendLatencyMarker(element.asLatencyMarker());
+			}
+			else {
+				throw new IOException("Unknown input record: " + element);
+			}
+		}
+
+		/**
+		 * Emit results from the finished head collector and its following finished ones.
+		 *
+		 * <p>NOTE: Since {@link #output(StreamElementEntry)} may be blocked if operator chain chained with
+		 * another {@link AsyncWaitOperator} and its buffer is full, we can not use an {@link Iterator} to
+		 * go through {@link #queue} because ConcurrentModificationException may be thrown while we remove
+		 * element in the queue by calling {@link Iterator#remove()}.
+		 *
+		 * <p>Example: Assume operator chain like this: async-wait-operator1(awo1) -> async-wait-operator2(awo2).
+		 * The buffer for awo1 is full so the main thread is blocked there.
+		 * The {@link Emitter} thread, named emitter1, in awo1 is outputting
+		 * data to awo2. Assume that 2 elements have been emitted and the buffer in awo1 has two vacancies. While
+		 * outputting the third one, the buffer in awo2 is full, so emitter1 will wait for a moment. If we use
+		 * {@link Iterator}, it is just before calling {@link Iterator#remove()}. Once the {@link #lock} is released
+		 * and luckily enough, the main thread get the lock. It will modify {@link #queue}, causing
+		 * ConcurrentModificationException once emitter1 runs to {@link Iterator#remove()}.
+		 *
+		 */
+		private void orderedProcess() throws Exception {
+			StreamElementEntry<OUT> entry;
+
+			while (queue.size() > 0 && (entry = queue.iterator().next()).isDone()) {
+				output(entry);
+
+				queue.remove(entry);
+			}
+		}
+
+		/**
+		 * Emit results for each finished collector. Try to emit results prior to the oldest watermark
+		 * in the buffer.
+		 * <p>
+		 * For example, assume the sequence of input StreamElements is:
+		 * Entry(ac1, record1) -> Entry(ac2, record2) -> Entry(ac3, watermark1) -> Entry(ac4, record3).
+		 * and both of ac2 and ac3 have finished. For unordered-mode, ac1 and ac2 are prior to watermark1,
+		 * so ac2 will be emitted. Since ac1 is not ready yet, ac3 have to wait until ac1 is done.
+		 */
+		private void unorderedProcess() throws Exception {
+			// try to emit finished AsyncCollectors in markerToFinishedEntries
+			if (markerToFinishedEntries.size() != 0) {
+				while (markerToFinishedEntries.size() != 0) {
+					Map.Entry<StreamElement, Set<StreamElementEntry<OUT>>> finishedStreamElementEntry =
+							markerToFinishedEntries.entrySet().iterator().next();
+
+					Set<StreamElementEntry<OUT>> finishedElementSet = finishedStreamElementEntry.getValue();
+
+					// While outputting results to the next operator, output may release lock if the following operator
+					// in the chain is another AsyncWaitOperator. During this period, there may be some
+					// finished StreamElementEntry coming into the finishedElementSet, and we should
+					// output all finished elements after re-acquiring the lock.
+					while (finishedElementSet.size() != 0) {
+						StreamElementEntry<OUT> finishedEntry = finishedElementSet.iterator().next();
+
+						output(finishedEntry);
+
+						queue.remove(finishedEntry);
+
+						entriesToMarkers.remove(finishedEntry);
+
+						finishedElementSet.remove(finishedEntry);
+					}
+
+					finishedStreamElementEntry.getValue().clear();
+
+
+					// if all StreamElements belonging to current Watermark / LatencyMarker have been emitted,
+					// emit current Watermark / LatencyMarker
+
+					if (queue.size() == 0) {
+						if (markerToFinishedEntries.size() != 0 || entriesToMarkers.size() != 0
+								|| lonelyEntries.size() != 0 || lonelyFinishedEntries.size() != 0) {
+							throw new IOException("Inner data info is not consistent.");
+						}
+					}
+					else {
+						// check the head AsyncCollector whether it is a Watermark or LatencyMarker.
+						StreamElementEntry<OUT> queueEntry = queue.iterator().next();
+
+						if (!queueEntry.isStreamRecord()) {
+							if (finishedStreamElementEntry.getKey() != queueEntry.getStreamElement()) {
+								throw new IOException("Watermark / LatencyMarker from finished collector map "
+									+ "and input buffer are not the same.");
+							}
+
+							output(queueEntry);
+
+							queue.remove(queueEntry);
+
+							// remove useless data in markerToFinishedEntries
+							markerToFinishedEntries.remove(finishedStreamElementEntry.getKey());
+						}
+						else {
+							break;
+						}
+					}
+				}
+			}
+
+			if (markerToFinishedEntries.size() == 0) {
+				// health check
+				if (entriesToMarkers.size() != 0) {
+					throw new IOException("Entries to marker map should be zero");
+				}
+
+				// no Watermark or LatencyMarker in the buffer yet, emit results in lonelyFinishedEntries
+				while (lonelyFinishedEntries.size() != 0) {
+					StreamElementEntry<OUT> entry = lonelyFinishedEntries.iterator().next();
+
+					output(entry);
+
+					queue.remove(entry);
+
+					lonelyEntries.remove(entry);
+
+					lonelyFinishedEntries.remove(entry);
+				}
+			}
+		}
+
+		private void processFinishedAsyncCollector() throws Exception {
+			if (mode == AsyncDataStream.OutputMode.ORDERED) {
+				orderedProcess();
+			} else {
+				unorderedProcess();
+			}
+		}
+
+		private void clearAndNotify() {
+			// clear all data
+			queue.clear();
+			entriesToMarkers.clear();
+			markerToFinishedEntries.clear();
+			lonelyEntries.clear();
+
+			running = false;
+
+			lock.notifyAll();
+		}
+
+		@Override
+		public void run() {
+			while (running) {
+				synchronized (lock) {
+
+					try {
+						while (!workwork) {
+							lock.wait();
+						}
+
+						processFinishedAsyncCollector();
+
+						lock.notifyAll();
+
+						workwork = false;
+					}
+					catch (InterruptedException e) {
+						// The source of InterruptedException is from:
+						//   1. lock.wait() statement in Emit
+						//   2. collector waiting for vacant buffer
+						// The action for this exception should try to clear all held data and
+						// exit Emit thread.
+
+						clearAndNotify();
+					}
+					catch (Exception e) {
+						// For exceptions, not InterruptedException, it should be propagated
+						// to main thread.
+						error = e;
+
+						clearAndNotify();
+					}
+				}
+			}
+		}
+
+		public void stop() {
+			running = false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java
new file mode 100644
index 0000000..1705c2d
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java
@@ -0,0 +1,36 @@
+/*
+ * 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.async.buffer;
+
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+
+/**
+ * {@link AsyncCollectorBuffer} entry for {@link LatencyMarker}
+ *
+ */
+public class LatencyMarkerEntry<OUT> extends AbstractBufferEntry<OUT> {
+	public LatencyMarkerEntry(LatencyMarker marker) {
+		super(marker);
+	}
+
+	@Override
+	public boolean isDone() {
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java
new file mode 100644
index 0000000..de7f606
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java
@@ -0,0 +1,82 @@
+/*
+ * 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.async.buffer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The base class for entries in the {@link AsyncCollectorBuffer}
+ *
+ * @param <OUT> Output data type
+ */
+
+@Internal
+public interface StreamElementEntry<OUT>  {
+	/**
+	 * Get result. Throw IOException while encountering an error.
+	 *
+	 * @return A List of result.
+	 * @throws IOException IOException wrapping errors from user codes.
+	 */
+	List<OUT> getResult() throws IOException;
+
+	/**
+	 * Set the internal flag, marking the async operator has finished.
+	 */
+	void markDone();
+
+	/**
+	 * Get the flag indicating the async operator has finished or not.
+	 *
+	 * @return True for finished async operator.
+	 */
+	boolean isDone();
+
+	/**
+	 * Check inner element is StreamRecord or not.
+	 *
+	 * @return True if element is StreamRecord.
+	 */
+	boolean isStreamRecord();
+
+	/**
+	 * Check inner element is Watermark or not.
+	 *
+	 * @return True if element is Watermark.
+	 */
+	boolean isWatermark();
+
+	/**
+	 * Check inner element is LatencyMarker or not.
+	 *
+	 * @return True if element is LatencyMarker.
+	 */
+	boolean isLatencyMarker();
+
+	/**
+	 * Get inner stream element.
+	 *
+	 * @return Inner {@link StreamElement}.
+	 */
+	StreamElement getStreamElement();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java
new file mode 100644
index 0000000..fb0dc3b
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java
@@ -0,0 +1,75 @@
+/*
+ * 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.async.buffer;
+
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * {@link AsyncCollectorBuffer} entry for {@link StreamRecord}
+ *
+ * @param <IN> Input data type
+ * @param <OUT> Output data type
+ */
+public class StreamRecordEntry<IN, OUT> extends AbstractBufferEntry<OUT> implements AsyncCollector<OUT> {
+	private List<OUT> result;
+	private Throwable error;
+
+	private boolean isDone = false;
+
+	private final AsyncCollectorBuffer<IN, OUT> buffer;
+
+	public StreamRecordEntry(StreamRecord<IN> element, AsyncCollectorBuffer<IN, OUT> buffer) {
+		super(element);
+		this.buffer = Preconditions.checkNotNull(buffer, "Reference to AsyncCollectorBuffer should not be null");
+	}
+
+	@Override
+	public void collect(List<OUT> result)  {
+		this.result = result;
+
+		this.buffer.markCollectorCompleted(this);
+	}
+
+	@Override
+	public void collect(Throwable error)  {
+		this.error = error;
+
+		this.buffer.markCollectorCompleted(this);
+	}
+
+	public List<OUT> getResult() throws IOException {
+		if (error != null) {
+			throw new IOException(error.getMessage());
+		}
+		return result;
+	}
+
+	public void markDone() {
+		isDone = true;
+	}
+
+	public boolean isDone() {
+		return isDone;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java
new file mode 100644
index 0000000..8883a2d
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java
@@ -0,0 +1,36 @@
+/*
+ * 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.async.buffer;
+
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * {@link AsyncCollectorBuffer} entry for {@link Watermark}
+ *
+ */
+public class WatermarkEntry<OUT> extends AbstractBufferEntry<OUT> {
+	public WatermarkEntry(Watermark watermark) {
+		super(watermark);
+	}
+
+	@Override
+	public boolean isDone() {
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
new file mode 100644
index 0000000..b2a58d2
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
@@ -0,0 +1,53 @@
+/*
+ * 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.async.collector;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.List;
+
+/**
+ * {@link AsyncCollector} collects data / error in user codes while processing async i/o.
+ *
+ * @param <OUT> Output type
+ */
+@Internal
+public interface AsyncCollector<OUT> {
+	/**
+	 * Set result.
+	 * <p>
+	 * Note that it should be called for exactly one time in the user code.
+	 * Calling this function for multiple times will cause data lose.
+	 * <p>
+	 * Put all results in a {@link List} and then issue {@link AsyncCollector#collect(List)}.
+	 * <p>
+	 * If the result is NULL, it will cause task fail. If collecting empty result set is allowable and
+	 * should not cause task fail-over, then try to collect an empty list collection.
+	 *
+	 * @param result A list of results.
+	 */
+	void collect(List<OUT> result);
+
+	/**
+	 * Set error
+	 *
+	 * @param error A Throwable object.
+	 */
+	void collect(Throwable error);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index f9b711e..1c27293 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -596,12 +596,12 @@ public abstract class AbstractStreamOperator<OUT>
 	}
 
 
-	protected void reportOrForwardLatencyMarker(LatencyMarker maker) {
+	protected void reportOrForwardLatencyMarker(LatencyMarker marker) {
 		// all operators are tracking latencies
-		this.latencyGauge.reportLatency(maker, false);
+		this.latencyGauge.reportLatency(marker, false);
 
 		// everything except sinks forwards latency markers
-		this.output.emitLatencyMarker(maker);
+		this.output.emitLatencyMarker(marker);
 	}
 
 	// ----------------------- Helper classes -----------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
new file mode 100644
index 0000000..9166865
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
@@ -0,0 +1,211 @@
+/*
+ * 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.async;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.streaming.api.functions.async.buffer.AsyncCollectorBuffer;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Iterator;
+
+/**
+ * The {@link AsyncWaitOperator} will accept input {@link StreamElement} from previous operators,
+ * pass them into {@link AsyncFunction}, make a snapshot for the inputs in the {@link AsyncCollectorBuffer}
+ * while checkpointing, and restore the {@link AsyncCollectorBuffer} from previous state.
+ * <p>
+ * Note that due to newly added working thread, named {@link AsyncCollectorBuffer.Emitter},
+ * if {@link AsyncWaitOperator} is chained with other operators, {@link StreamTask} has to make sure that
+ * the the order to open operators in the operator chain should be from the tail operator to the head operator,
+ * and order to close operators in the operator chain should be from the head operator to the tail operator.
+ *
+ * @param <IN> Input type for the operator.
+ * @param <OUT> Output type for the operator.
+ */
+@Internal
+public class AsyncWaitOperator<IN, OUT>
+	extends AbstractUdfStreamOperator<OUT, AsyncFunction<IN, OUT>>
+	implements OneInputStreamOperator<IN, OUT>
+{
+	private static final long serialVersionUID = 1L;
+
+	private final static String STATE_NAME = "_async_wait_operator_state_";
+
+	/**
+	 * {@link TypeSerializer} for inputs while making snapshots.
+	 */
+	private transient StreamElementSerializer<IN> inStreamElementSerializer;
+
+	/**
+	 * input stream elements from the state
+	 */
+	private transient ListState<StreamElement> recoveredStreamElements;
+
+	private transient TimestampedCollector<OUT> collector;
+
+	private transient AsyncCollectorBuffer<IN, OUT> buffer;
+
+	/**
+	 * Checkpoint lock from {@link StreamTask#lock}
+	 */
+	private transient Object checkpointLock;
+
+	private final int bufferSize;
+	private final AsyncDataStream.OutputMode mode;
+
+
+	public AsyncWaitOperator(AsyncFunction<IN, OUT> asyncFunction, int bufferSize, AsyncDataStream.OutputMode mode) {
+		super(asyncFunction);
+		chainingStrategy = ChainingStrategy.ALWAYS;
+
+		Preconditions.checkArgument(bufferSize > 0, "The number of concurrent async operation should be greater than 0.");
+		this.bufferSize = bufferSize;
+
+		this.mode = mode;
+	}
+
+	@Override
+	public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output) {
+		super.setup(containingTask, config, output);
+
+		this.inStreamElementSerializer =
+				new StreamElementSerializer(this.getOperatorConfig().<IN>getTypeSerializerIn1(getUserCodeClassloader()));
+
+		this.collector = new TimestampedCollector<>(output);
+
+		this.checkpointLock = containingTask.getCheckpointLock();
+
+		this.buffer = new AsyncCollectorBuffer<>(bufferSize, mode, output, collector, this.checkpointLock, this);
+	}
+
+	@Override
+	public void open() throws Exception {
+		super.open();
+
+		// process stream elements from state, since the Emit thread will start soon as all elements from
+		// previous state are in the AsyncCollectorBuffer, we have to make sure that the order to open all
+		// operators in the operator chain should be from the tail operator to the head operator.
+		if (this.recoveredStreamElements != null) {
+			for (StreamElement element : this.recoveredStreamElements.get()) {
+				if (element.isRecord()) {
+					processElement(element.<IN>asRecord());
+				}
+				else if (element.isWatermark()) {
+					processWatermark(element.asWatermark());
+				}
+				else if (element.isLatencyMarker()) {
+					processLatencyMarker(element.asLatencyMarker());
+				}
+				else {
+					throw new Exception("Unknown record type: "+element.getClass());
+				}
+			}
+			this.recoveredStreamElements = null;
+		}
+
+		buffer.startEmitterThread();
+	}
+
+	@Override
+	public void processElement(StreamRecord<IN> element) throws Exception {
+		AsyncCollector<OUT> collector = buffer.addStreamRecord(element);
+
+		userFunction.asyncInvoke(element.getValue(), collector);
+	}
+
+	@Override
+	public void processWatermark(Watermark mark) throws Exception {
+		buffer.addWatermark(mark);
+	}
+
+	@Override
+	public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception {
+		buffer.addLatencyMarker(latencyMarker);
+	}
+
+	@Override
+	public void snapshotState(StateSnapshotContext context) throws Exception {
+		super.snapshotState(context);
+
+		ListState<StreamElement> partitionableState =
+				getOperatorStateBackend().getOperatorState(new ListStateDescriptor<>(STATE_NAME, inStreamElementSerializer));
+		partitionableState.clear();
+
+		Iterator<StreamElement> iterator = buffer.getStreamElementsInBuffer();
+		while (iterator.hasNext()) {
+			partitionableState.add(iterator.next());
+		}
+	}
+
+	@Override
+	public void initializeState(StateInitializationContext context) throws Exception {
+		recoveredStreamElements =
+				context.getOperatorStateStore().getOperatorState(new ListStateDescriptor<>(STATE_NAME, inStreamElementSerializer));
+
+	}
+
+	@Override
+	public void close() throws Exception {
+		try {
+			buffer.waitEmpty();
+		}
+		finally {
+			// make sure Emitter thread exits and close user function
+			buffer.stopEmitterThread();
+
+			super.close();
+		}
+	}
+
+	@Override
+	public void dispose() throws Exception {
+		super.dispose();
+
+		buffer.stopEmitterThread();
+	}
+
+	public void sendLatencyMarker(LatencyMarker marker) throws Exception {
+		super.processLatencyMarker(marker);
+	}
+
+	@VisibleForTesting
+	public AsyncCollectorBuffer<IN, OUT> getBuffer() {
+		return buffer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
index 5ea84fb..680cc29 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
@@ -17,13 +17,6 @@
 
 package org.apache.flink.streaming.runtime.tasks;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -37,22 +30,28 @@ import org.apache.flink.runtime.plugable.SerializationDelegate;
 import org.apache.flink.streaming.api.collector.selector.CopyingDirectedOutput;
 import org.apache.flink.streaming.api.collector.selector.DirectedOutput;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.graph.StreamEdge;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
 import org.apache.flink.streaming.runtime.io.StreamRecordWriter;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
 import org.apache.flink.util.XORShiftRandom;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
 
 /**
  * The {@code OperatorChain} contains all operators that are executed as one chain within a single
@@ -67,7 +66,7 @@ public class OperatorChain<OUT, OP extends StreamOperator<OUT>> {
 	private static final Logger LOG = LoggerFactory.getLogger(OperatorChain.class);
 	
 	private final StreamOperator<?>[] allOperators;
-	
+
 	private final RecordWriterOutput<?>[] streamOutputs;
 	
 	private final Output<StreamRecord<OUT>> chainEntryPoint;
@@ -96,7 +95,6 @@ public class OperatorChain<OUT, OP extends StreamOperator<OUT>> {
 		try {
 			for (int i = 0; i < outEdgesInOrder.size(); i++) {
 				StreamEdge outEdge = outEdgesInOrder.get(i);
-				
 				RecordWriterOutput<?> streamOutput = createStreamOutput(
 						outEdge, chainedConfigs.get(outEdge.getSourceId()), i,
 						containingTask.getEnvironment(), containingTask.getName());
@@ -116,7 +114,7 @@ public class OperatorChain<OUT, OP extends StreamOperator<OUT>> {
 
 			// add head operator to end of chain
 			allOps.add(headOperator);
-			
+
 			this.allOperators = allOps.toArray(new StreamOperator<?>[allOps.size()]);
 			
 			success = true;
@@ -245,7 +243,6 @@ public class OperatorChain<OUT, OP extends StreamOperator<OUT>> {
 
 			Output<StreamRecord<T>> output = createChainedOperator(
 					containingTask, chainedOpConfig, chainedConfigs, userCodeClassloader, streamOutputs, allOperators);
-			
 			allOutputs.add(new Tuple2<>(output, outputEdge));
 		}
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/f5283076/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 88a29ab..0fb22b8 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -773,8 +773,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 					keyGroupRange,
 					restoreStateHandles.getManagedKeyedState(),
 					getEnvironment().getTaskKvStateRegistry());
-
-			restoreStateHandles = null; // GC friendliness
 		} else {
 			keyedStateBackend = stateBackend.createKeyedStateBackend(
 					getEnvironment(),
@@ -1076,6 +1074,7 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 					outStream.close();
 				}
 			}
+
 			nonPartitionedStates.add(stateHandle);
 		}
 


[6/7] flink git commit: [FLINK-4391] Add timeout parameter for asynchronous I/O

Posted by tr...@apache.org.
[FLINK-4391] Add timeout parameter for asynchronous I/O

The timeout defines how long an asynchronous I/O operation can take. If the operation
takes longer than the timeout, then it is failed with an TimeoutException.

Annotate classes with internal Annotation


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6c5a8711
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6c5a8711
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6c5a8711

Branch: refs/heads/master
Commit: 6c5a8711d80dfcea20967aea009bac51122d5094
Parents: ad603d5
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Dec 20 03:42:25 2016 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Dec 20 05:04:51 2016 +0100

----------------------------------------------------------------------
 .../examples/async/AsyncIOExample.java          |  19 ++-
 .../api/datastream/AsyncDataStream.java         |  67 +++++++---
 .../api/operators/async/AsyncWaitOperator.java  |   5 +-
 .../streaming/api/operators/async/Emitter.java  |   2 +
 .../api/operators/async/OperatorActions.java    |   2 +
 .../async/queue/AsyncCollectionResult.java      |   3 +
 .../api/operators/async/queue/AsyncResult.java  |   2 +
 .../async/queue/AsyncWatermarkResult.java       |   2 +
 .../async/queue/OrderedStreamElementQueue.java  |   2 +
 .../async/queue/StreamElementQueue.java         |   2 +
 .../async/queue/StreamElementQueueEntry.java    |   2 +
 .../async/queue/StreamRecordQueueEntry.java     |   2 +
 .../queue/UnorderedStreamElementQueue.java      |   2 +
 .../async/queue/WatermarkQueueEntry.java        |   2 +
 .../operators/async/AsyncWaitOperatorTest.java  | 131 +++++++++++++++++--
 .../util/AbstractStreamOperatorTestHarness.java |  42 ++++--
 .../util/OneInputStreamOperatorTestHarness.java |  21 +++
 .../streaming/api/StreamingOperatorsITCase.java |  17 ++-
 18 files changed, 277 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
index 6dde537..2b05983 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
@@ -206,7 +206,8 @@ public class AsyncIOExample {
 				"[--maxCount <max number of input from source, -1 for infinite input>] " +
 				"[--sleepFactor <interval to sleep for each stream element>] [--failRatio <possibility to throw exception>] " +
 				"[--waitMode <ordered or unordered>] [--waitOperatorParallelism <parallelism for async wait operator>] " +
-				"[--eventType <EventTime or IngestionTime>] [--shutdownWaitTS <milli sec to wait for thread pool>]");
+				"[--eventType <EventTime or IngestionTime>] [--shutdownWaitTS <milli sec to wait for thread pool>]" +
+				"[--timeout <Timeout for the asynchronous operations>]");
 	}
 
 	public static void main(String[] args) throws Exception {
@@ -226,6 +227,7 @@ public class AsyncIOExample {
 		final int taskNum;
 		final String timeType;
 		final long shutdownWaitTS;
+		final long timeout;
 
 		try {
 			// check the configuration for the job
@@ -238,6 +240,7 @@ public class AsyncIOExample {
 			taskNum = params.getInt("waitOperatorParallelism", 1);
 			timeType = params.get("eventType", "EventTime");
 			shutdownWaitTS = params.getLong("shutdownWaitTS", 20000);
+			timeout = params.getLong("timeout", 10000L);
 		} catch (Exception e) {
 			printUsage();
 
@@ -292,10 +295,20 @@ public class AsyncIOExample {
 		// add async operator to streaming job
 		DataStream<String> result;
 		if (ORDERED.equals(mode)) {
-			result = AsyncDataStream.orderedWait(inputStream, function, 20).setParallelism(taskNum);
+			result = AsyncDataStream.orderedWait(
+				inputStream,
+				function,
+				timeout,
+				TimeUnit.MILLISECONDS,
+				20).setParallelism(taskNum);
 		}
 		else {
-			result = AsyncDataStream.unorderedWait(inputStream, function, 20).setParallelism(taskNum);
+			result = AsyncDataStream.unorderedWait(
+				inputStream,
+				function,
+				timeout,
+				TimeUnit.MILLISECONDS,
+				20).setParallelism(taskNum);
 		}
 
 		// add a reduce to get the sum of each keys.

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java
index 4fefde0..8132d28 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java
@@ -24,6 +24,8 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.streaming.api.functions.async.AsyncFunction;
 import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * A helper class to apply {@link AsyncFunction} to a data stream.
  * <p>
@@ -31,7 +33,7 @@ import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
  * DataStream<String> input = ...
  * AsyncFunction<String, Tuple<String, String>> asyncFunc = ...
  *
- * AsyncDataStream.orderedWait(input, asyncFunc, 100);
+ * AsyncDataStream.orderedWait(input, asyncFunc, timeout, TimeUnit.MILLISECONDS, 100);
  * }
  * </pre>
  */
@@ -40,13 +42,14 @@ import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
 public class AsyncDataStream {
 	public enum OutputMode { ORDERED, UNORDERED }
 
-	private static final int DEFAULT_BUFFER_SIZE = 100;
+	private static final int DEFAULT_QUEUE_CAPACITY = 100;
 
 	/**
 	 * Add an AsyncWaitOperator.
 	 *
 	 * @param in The {@link DataStream} where the {@link AsyncWaitOperator} will be added.
 	 * @param func {@link AsyncFunction} wrapped inside {@link AsyncWaitOperator}.
+	 * @param timeout for the asynchronous operation to complete
 	 * @param bufSize The max number of inputs the {@link AsyncWaitOperator} can hold inside.
 	 * @param mode Processing mode for {@link AsyncWaitOperator}.
 	 * @param <IN> Input type.
@@ -56,6 +59,7 @@ public class AsyncDataStream {
 	private static <IN, OUT> SingleOutputStreamOperator<OUT> addOperator(
 			DataStream<IN> in,
 			AsyncFunction<IN, OUT> func,
+			long timeout,
 			int bufSize,
 			OutputMode mode) {
 
@@ -64,8 +68,11 @@ public class AsyncDataStream {
 				true, in.getType(), Utils.getCallLocationName(), true);
 
 		// create transform
-		AsyncWaitOperator<IN, OUT> operator =
-				new AsyncWaitOperator<>(in.getExecutionEnvironment().clean(func), bufSize, mode);
+		AsyncWaitOperator<IN, OUT> operator = new AsyncWaitOperator<>(
+			in.getExecutionEnvironment().clean(func),
+			timeout,
+			bufSize,
+			mode);
 
 		return in.transform("async wait operator", outTypeInfo, operator);
 	}
@@ -75,7 +82,9 @@ public class AsyncDataStream {
 	 *
 	 * @param in Input {@link DataStream}
 	 * @param func {@link AsyncFunction}
-	 * @param bufSize The max number of async i/o operation that can be triggered
+	 * @param timeout for the asynchronous operation to complete
+	 * @param timeUnit of the given timeout
+	 * @param capacity The max number of async i/o operation that can be triggered
 	 * @param <IN> Type of input record
 	 * @param <OUT> Type of output record
 	 * @return A new {@link SingleOutputStreamOperator}.
@@ -83,30 +92,44 @@ public class AsyncDataStream {
 	public static <IN, OUT> SingleOutputStreamOperator<OUT> unorderedWait(
 			DataStream<IN> in,
 			AsyncFunction<IN, OUT> func,
-			int bufSize) {
-		return addOperator(in, func, bufSize, OutputMode.UNORDERED);
+			long timeout,
+			TimeUnit timeUnit,
+			int capacity) {
+		return addOperator(in, func, timeUnit.toMillis(timeout), capacity, OutputMode.UNORDERED);
 	}
 
 	/**
 	 * Add an AsyncWaitOperator. The order of output stream records may be reordered.
 	 * @param in Input {@link DataStream}
 	 * @param func {@link AsyncFunction}
+	 * @param timeout for the asynchronous operation to complete
+	 * @param timeUnit of the given timeout
 	 * @param <IN> Type of input record
 	 * @param <OUT> Type of output record
 	 * @return A new {@link SingleOutputStreamOperator}.
 	 */
 	public static <IN, OUT> SingleOutputStreamOperator<OUT> unorderedWait(
 			DataStream<IN> in,
-			AsyncFunction<IN, OUT> func) {
-		return addOperator(in, func, DEFAULT_BUFFER_SIZE, OutputMode.UNORDERED);
+			AsyncFunction<IN, OUT> func,
+			long timeout,
+			TimeUnit timeUnit) {
+		return addOperator(
+			in,
+			func,
+			timeUnit.toMillis(timeout),
+			DEFAULT_QUEUE_CAPACITY,
+			OutputMode.UNORDERED);
 	}
 
 	/**
-	 * Add an AsyncWaitOperator. The order to process input records is guaranteed to be the same as input ones.
+	 * Add an AsyncWaitOperator. The order to process input records is guaranteed to be the same as
+	 * input ones.
 	 *
 	 * @param in Input {@link DataStream}
 	 * @param func {@link AsyncFunction}
-	 * @param bufSize The max number of async i/o operation that can be triggered
+	 * @param timeout for the asynchronous operation to complete
+	 * @param timeUnit of the given timeout
+	 * @param capacity The max number of async i/o operation that can be triggered
 	 * @param <IN> Type of input record
 	 * @param <OUT> Type of output record
 	 * @return A new {@link SingleOutputStreamOperator}.
@@ -114,22 +137,34 @@ public class AsyncDataStream {
 	public static <IN, OUT> SingleOutputStreamOperator<OUT> orderedWait(
 			DataStream<IN> in,
 			AsyncFunction<IN, OUT> func,
-			int bufSize) {
-		return addOperator(in, func, bufSize, OutputMode.ORDERED);
+			long timeout,
+			TimeUnit timeUnit,
+			int capacity) {
+		return addOperator(in, func, timeUnit.toMillis(timeout), capacity, OutputMode.ORDERED);
 	}
 
 	/**
-	 * Add an AsyncWaitOperator. The order to process input records is guaranteed to be the same as input ones.
+	 * Add an AsyncWaitOperator. The order to process input records is guaranteed to be the same as
+	 * input ones.
 	 *
 	 * @param in Input {@link DataStream}
 	 * @param func {@link AsyncFunction}
+	 * @param timeout for the asynchronous operation to complete
+	 * @param timeUnit of the given timeout
 	 * @param <IN> Type of input record
 	 * @param <OUT> Type of output record
 	 * @return A new {@link SingleOutputStreamOperator}.
 	 */
 	public static <IN, OUT> SingleOutputStreamOperator<OUT> orderedWait(
 			DataStream<IN> in,
-			AsyncFunction<IN, OUT> func) {
-		return addOperator(in, func, DEFAULT_BUFFER_SIZE, OutputMode.ORDERED);
+			AsyncFunction<IN, OUT> func,
+			long timeout,
+			TimeUnit timeUnit) {
+		return addOperator(
+			in,
+			func,
+			timeUnit.toMillis(timeout),
+			DEFAULT_QUEUE_CAPACITY,
+			OutputMode.ORDERED);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
index 88fc833..754b754 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
@@ -114,6 +114,7 @@ public class AsyncWaitOperator<IN, OUT>
 
 	public AsyncWaitOperator(
 			AsyncFunction<IN, OUT> asyncFunction,
+			long timeout,
 			int capacity,
 			AsyncDataStream.OutputMode outputMode) {
 		super(asyncFunction);
@@ -124,7 +125,7 @@ public class AsyncWaitOperator<IN, OUT>
 
 		this.outputMode = Preconditions.checkNotNull(outputMode, "outputMode");
 
-		this.timeout = -1L;
+		this.timeout = timeout;
 	}
 
 	@Override
@@ -200,7 +201,7 @@ public class AsyncWaitOperator<IN, OUT>
 
 		if (timeout > 0L) {
 			// register a timeout for this AsyncStreamRecordBufferEntry
-			long timeoutTimestamp = timeout + System.currentTimeMillis();
+			long timeoutTimestamp = timeout + getProcessingTimeService().getCurrentProcessingTime();
 
 			getProcessingTimeService().registerTimer(
 				timeoutTimestamp,

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
index 4b22aaa..c122d23 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.operators.async.queue.AsyncCollectionResult;
@@ -37,6 +38,7 @@ import java.util.Collection;
  *
  * @param <OUT> Type of the output elements
  */
+@Internal
 public class Emitter<OUT> implements Runnable {
 
 	private static final Logger LOG = LoggerFactory.getLogger(Emitter.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java
index 5a2e43c..916b412 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java
@@ -18,11 +18,13 @@
 
 package org.apache.flink.streaming.api.operators.async;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 
 /**
  * Interface for {@link StreamOperator} actions.
  */
+@Internal
 public interface OperatorActions {
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java
index 8088bf0..6226ae6 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
+
 import java.util.Collection;
 
 /**
@@ -25,6 +27,7 @@ import java.util.Collection;
  *
  * @param <T> Type of the collection elements.
  */
+@Internal
 public interface AsyncCollectionResult<T> extends AsyncResult {
 
 	boolean hasTimestamp();

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java
index 1a99928..751de76 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.streaming.api.functions.async.AsyncFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 
@@ -26,6 +27,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
  * either be a {@link Watermark} or a collection of new output elements produced by the
  * {@link AsyncFunction}.
  */
+@Internal
 public interface AsyncResult {
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java
index c19b520..68bde3a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java
@@ -18,11 +18,13 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.streaming.api.watermark.Watermark;
 
 /**
  * {@link AsyncResult} subclass for asynchronous result {@link Watermark}.
  */
+@Internal
 public interface AsyncWatermarkResult extends AsyncResult {
 	/**
 	 * Get the resulting watermark.

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java
index 2bbcb6c..414b3c0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.concurrent.AcceptFunction;
 import org.apache.flink.streaming.api.operators.async.OperatorActions;
 import org.apache.flink.util.Preconditions;
@@ -37,6 +38,7 @@ import java.util.concurrent.locks.ReentrantLock;
  * to the queue. Thus, even if the completion order can be arbitrary, the output order strictly
  * follows the insertion order (element cannot overtake each other).
  */
+@Internal
 public class OrderedStreamElementQueue implements StreamElementQueue {
 
 	private static final Logger LOG = LoggerFactory.getLogger(OrderedStreamElementQueue.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java
index 1a2c4a8..e02b8b0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
 
 import java.util.Collection;
@@ -25,6 +26,7 @@ import java.util.Collection;
 /**
  * Interface for blocking stream element queues for the {@link AsyncWaitOperator}.
  */
+@Internal
 public interface StreamElementQueue {
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
index 06ebf3c..7db9d4f 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.concurrent.AcceptFunction;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
@@ -32,6 +33,7 @@ import java.util.concurrent.Executor;
  *
  * @param <T> Type of the result
  */
+@Internal
 public abstract class StreamElementQueueEntry<T> implements AsyncResult {
 
 	/** Stream element */

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java
index f0e707e..c654702 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.concurrent.CompletableFuture;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
@@ -34,6 +35,7 @@ import java.util.Collection;
  *
  * @param <OUT> Type of the asynchronous collection result
  */
+@Internal
 public class StreamRecordQueueEntry<OUT> extends StreamElementQueueEntry<Collection<OUT>>
 	implements AsyncCollectionResult<OUT>, AsyncCollector<OUT> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
index 603d8cc..f244008 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.concurrent.AcceptFunction;
 import org.apache.flink.streaming.api.operators.async.OperatorActions;
 import org.apache.flink.util.Preconditions;
@@ -41,6 +42,7 @@ import java.util.concurrent.locks.ReentrantLock;
  * and no watermark can overtake a stream record. However, stream records falling in the same
  * segment between two watermarks can overtake each other (their emission order is not guaranteed).
  */
+@Internal
 public class UnorderedStreamElementQueue implements StreamElementQueue {
 
 	private static final Logger LOG = LoggerFactory.getLogger(UnorderedStreamElementQueue.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java
index 6fe4f44..1f48303 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.api.operators.async.queue;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.streaming.api.watermark.Watermark;
@@ -25,6 +26,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 /**
  * {@link StreamElementQueueEntry} implementation for the {@link Watermark}.
  */
+@Internal
 public class WatermarkQueueEntry extends StreamElementQueueEntry<Watermark> implements AsyncWatermarkResult {
 
 	private final Future<Watermark> future;

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
index 10ee14f..d9e885e 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.api.operators.async;
 
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -27,11 +28,15 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.SubtaskState;
+import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
 import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
+import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
 import org.apache.flink.runtime.state.TaskStateHandles;
+import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.datastream.AsyncDataStream;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -51,6 +56,7 @@ import org.apache.flink.streaming.util.TestHarnessUtil;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
 
 import java.util.ArrayDeque;
 import java.util.Collections;
@@ -59,11 +65,16 @@ import java.util.Iterator;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests for {@link AsyncWaitOperator}. These test that:
@@ -77,10 +88,12 @@ import static org.junit.Assert.assertEquals;
  */
 public class AsyncWaitOperatorTest extends TestLogger {
 
+	private static final long TIMEOUT = 1000L;
+
 	private static class MyAsyncFunction extends RichAsyncFunction<Integer, Integer> {
 		private static final long serialVersionUID = 8522411971886428444L;
 
-		private static final long TIMEOUT = 5000L;
+		private static final long TERMINATION_TIMEOUT = 5000L;
 		private static final int THREAD_POOL_SIZE = 10;
 
 		static ExecutorService executorService;
@@ -114,7 +127,7 @@ public class AsyncWaitOperatorTest extends TestLogger {
 					executorService.shutdown();
 
 					try {
-						if (!executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS)) {
+						if (!executorService.awaitTermination(TERMINATION_TIMEOUT, TimeUnit.MILLISECONDS)) {
 							executorService.shutdownNow();
 						}
 					} catch (InterruptedException interrupted) {
@@ -219,7 +232,11 @@ public class AsyncWaitOperatorTest extends TestLogger {
 	}
 
 	private void testEventTime(AsyncDataStream.OutputMode mode) throws Exception {
-		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(new MyAsyncFunction(), 2, mode);
+		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(
+			new MyAsyncFunction(),
+			TIMEOUT,
+			2,
+			mode);
 
 		final OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
 				new OneInputStreamOperatorTestHarness<>(operator, IntSerializer.INSTANCE);
@@ -280,7 +297,8 @@ public class AsyncWaitOperatorTest extends TestLogger {
 	}
 
 	private void testProcessingTime(AsyncDataStream.OutputMode mode) throws Exception {
-		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(new MyAsyncFunction(), 6, mode);
+		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(
+			new MyAsyncFunction(), TIMEOUT, 6, mode);
 
 		final OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator, IntSerializer.INSTANCE);
 
@@ -380,10 +398,20 @@ public class AsyncWaitOperatorTest extends TestLogger {
 		DataStream<Integer> input = chainEnv.fromElements(1, 2, 3);
 
 		if (withLazyFunction) {
-			input = AsyncDataStream.orderedWait(input, new LazyAsyncFunction(), 6);
+			input = AsyncDataStream.orderedWait(
+				input,
+				new LazyAsyncFunction(),
+				TIMEOUT,
+				TimeUnit.MILLISECONDS,
+				6);
 		}
 		else {
-			input = AsyncDataStream.orderedWait(input, new MyAsyncFunction(), 6);
+			input = AsyncDataStream.orderedWait(
+				input,
+				new MyAsyncFunction(),
+				TIMEOUT,
+				TimeUnit.MILLISECONDS,
+				6);
 		}
 
 		// the map function is designed to chain after async function. we place an Integer object in it and
@@ -407,7 +435,12 @@ public class AsyncWaitOperatorTest extends TestLogger {
 			}
 		});
 
-		input = AsyncDataStream.unorderedWait(input, new MyAsyncFunction(), 3);
+		input = AsyncDataStream.unorderedWait(
+			input,
+			new MyAsyncFunction(),
+			TIMEOUT,
+			TimeUnit.MILLISECONDS,
+			3);
 
 		input.map(new MapFunction<Integer, Integer>() {
 			private static final long serialVersionUID = 5162085254238405527L;
@@ -432,8 +465,11 @@ public class AsyncWaitOperatorTest extends TestLogger {
 		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
 				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
 
-		AsyncWaitOperator<Integer, Integer> operator =
-				new AsyncWaitOperator<>(new LazyAsyncFunction(), 3, AsyncDataStream.OutputMode.ORDERED);
+		AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(
+			new LazyAsyncFunction(),
+			TIMEOUT,
+			3,
+			AsyncDataStream.OutputMode.ORDERED);
 
 		final StreamConfig streamConfig = testHarness.getStreamConfig();
 		streamConfig.setStreamOperator(operator);
@@ -481,8 +517,11 @@ public class AsyncWaitOperatorTest extends TestLogger {
 		final OneInputStreamTaskTestHarness<Integer, Integer> restoredTaskHarness =
 				new OneInputStreamTaskTestHarness<>(restoredTask, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
 
-		AsyncWaitOperator<Integer, Integer> restoredOperator =
-				new AsyncWaitOperator<>(new MyAsyncFunction(), 6, AsyncDataStream.OutputMode.ORDERED);
+		AsyncWaitOperator<Integer, Integer> restoredOperator = new AsyncWaitOperator<>(
+			new MyAsyncFunction(),
+			TIMEOUT,
+			6,
+			AsyncDataStream.OutputMode.ORDERED);
 
 		restoredTaskHarness.getStreamConfig().setStreamOperator(restoredOperator);
 
@@ -561,4 +600,74 @@ public class AsyncWaitOperatorTest extends TestLogger {
 			return checkpointStateHandles;
 		}
 	}
+
+	@Test
+	public void testAsyncTimeout() throws Exception {
+		final long timeout = 10L;
+
+		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(
+			new LazyAsyncFunction(),
+			timeout,
+			2,
+			AsyncDataStream.OutputMode.ORDERED);
+
+		final Environment mockEnvironment = mock(Environment.class);
+
+		final Configuration taskConfiguration = new Configuration();
+		final ExecutionConfig executionConfig = new ExecutionConfig();
+		final TaskMetricGroup metricGroup = new UnregisteredTaskMetricsGroup();
+		final Configuration taskManagerConfiguration = new Configuration();
+		final TaskManagerRuntimeInfo taskManagerRuntimeInfo = new TaskManagerRuntimeInfo("localhost", taskManagerConfiguration, "/tmp");
+		final TaskInfo taskInfo = new TaskInfo("foobarTask", 1, 0, 1, 1);
+
+		when(mockEnvironment.getTaskConfiguration()).thenReturn(taskConfiguration);
+		when(mockEnvironment.getExecutionConfig()).thenReturn(executionConfig);
+		when(mockEnvironment.getMetricGroup()).thenReturn(metricGroup);
+		when(mockEnvironment.getTaskManagerInfo()).thenReturn(taskManagerRuntimeInfo);
+		when(mockEnvironment.getTaskInfo()).thenReturn(taskInfo);
+		when(mockEnvironment.getUserClassLoader()).thenReturn(AsyncWaitOperatorTest.class.getClassLoader());
+
+		final OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
+			new OneInputStreamOperatorTestHarness<>(operator, IntSerializer.INSTANCE, mockEnvironment);
+
+		final long initialTime = 0L;
+		final ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(initialTime);
+
+		synchronized (testHarness.getCheckpointLock()) {
+			testHarness.processElement(new StreamRecord<>(1, initialTime));
+			testHarness.setProcessingTime(initialTime + 5L);
+			testHarness.processElement(new StreamRecord<>(2, initialTime + 5L));
+		}
+
+		// trigger the timeout of the first stream record
+		testHarness.setProcessingTime(initialTime + timeout + 1L);
+
+		// allow the second async stream record to be processed
+		LazyAsyncFunction.countDown();
+
+		// wait until all async collectors in the buffer have been emitted out.
+		synchronized (testHarness.getCheckpointLock()) {
+			testHarness.close();
+		}
+
+		expectedOutput.add(new StreamRecord<>(2, initialTime + 5L));
+
+		TestHarnessUtil.assertOutputEquals("Output with watermark was not correct.", expectedOutput, testHarness.getOutput());
+
+		ArgumentCaptor<Throwable> argumentCaptor = ArgumentCaptor.forClass(Throwable.class);
+
+		verify(mockEnvironment).failExternally(argumentCaptor.capture());
+
+		Throwable failureCause = argumentCaptor.getValue();
+
+		Assert.assertNotNull(failureCause.getCause());
+		Assert.assertTrue(failureCause.getCause() instanceof ExecutionException);
+
+		Assert.assertNotNull(failureCause.getCause().getCause());
+		Assert.assertTrue(failureCause.getCause().getCause() instanceof TimeoutException);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
index 830cd6f..b623fa1 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
@@ -54,6 +54,7 @@ import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
 import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
 import org.apache.flink.util.FutureUtil;
+import org.apache.flink.util.Preconditions;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -107,29 +108,44 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 	private volatile boolean wasFailedExternally = false;
 
 	public AbstractStreamOperatorTestHarness(
+		StreamOperator<OUT> operator,
+		int maxParallelism,
+		int numSubtasks,
+		int subtaskIndex) throws Exception {
+
+		this(
+			operator,
+			maxParallelism,
+			numSubtasks,
+			subtaskIndex,
+			new MockEnvironment(
+				"MockTask",
+				3 * 1024 * 1024,
+				new MockInputSplitProvider(),
+				1024,
+				new Configuration(),
+				new ExecutionConfig(),
+				maxParallelism,
+				numSubtasks,
+				subtaskIndex));
+	}
+
+	public AbstractStreamOperatorTestHarness(
 			StreamOperator<OUT> operator,
 			int maxParallelism,
 			int numSubtasks,
-			int subtaskIndex) throws Exception {
+			int subtaskIndex,
+			final Environment environment) throws Exception {
 		this.operator = operator;
 		this.outputList = new ConcurrentLinkedQueue<>();
-		Configuration underlyingConfig = new Configuration();
+		Configuration underlyingConfig = environment.getTaskConfiguration();
 		this.config = new StreamConfig(underlyingConfig);
 		this.config.setCheckpointingEnabled(true);
-		this.executionConfig = new ExecutionConfig();
+		this.executionConfig = environment.getExecutionConfig();
 		this.closableRegistry = new CloseableRegistry();
 		this.checkpointLock = new Object();
 
-		environment = new MockEnvironment(
-				"MockTask",
-				3 * 1024 * 1024,
-				new MockInputSplitProvider(),
-				1024,
-				underlyingConfig,
-				executionConfig,
-				maxParallelism,
-				numSubtasks,
-				subtaskIndex);
+		this.environment = Preconditions.checkNotNull(environment);
 
 		mockTask = mock(StreamTask.class);
 		processingTimeService = new TestProcessingTimeService();

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index 86fbaa0..9b02378 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.util;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -45,6 +46,15 @@ public class OneInputStreamOperatorTestHarness<IN, OUT>
 		config.setTypeSerializerIn1(Preconditions.checkNotNull(typeSerializerIn));
 	}
 
+	public OneInputStreamOperatorTestHarness(
+		OneInputStreamOperator<IN, OUT> operator,
+		TypeSerializer<IN> typeSerializerIn,
+		Environment environment) throws Exception {
+		this(operator, 1, 1, 0, environment);
+
+		config.setTypeSerializerIn1(Preconditions.checkNotNull(typeSerializerIn));
+	}
+
 	public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator) throws Exception {
 		this(operator, 1, 1, 0);
 	}
@@ -59,6 +69,17 @@ public class OneInputStreamOperatorTestHarness<IN, OUT>
 		this.oneInputOperator = operator;
 	}
 
+	public OneInputStreamOperatorTestHarness(
+		OneInputStreamOperator<IN, OUT> operator,
+		int maxParallelism,
+		int numTubtasks,
+		int subtaskIndex,
+		Environment environment) throws Exception {
+		super(operator, maxParallelism, numTubtasks, subtaskIndex, environment);
+
+		this.oneInputOperator = operator;
+	}
+
 	public void processElement(IN value, long timestamp) throws Exception {
 		processElement(new StreamRecord<>(value, timestamp));
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/6c5a8711/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
index 3631965..8ea1bd8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
@@ -41,6 +41,7 @@ import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase {
 
@@ -206,6 +207,7 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 	@Test
 	public void testAsyncWaitOperator() throws Exception {
 		final int numElements = 5;
+		final long timeout = 1000L;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -240,7 +242,12 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 			}
 		};
 
-		DataStream<Integer> orderedResult = AsyncDataStream.orderedWait(input, function, 2).setParallelism(1);
+		DataStream<Integer> orderedResult = AsyncDataStream.orderedWait(
+			input,
+			function,
+			timeout,
+			TimeUnit.MILLISECONDS,
+			2).setParallelism(1);
 
 		// save result from ordered process
 		final MemorySinkFunction sinkFunction1 = new MemorySinkFunction(0);
@@ -249,8 +256,12 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		orderedResult.addSink(sinkFunction1).setParallelism(1);
 
-
-		DataStream<Integer> unorderedResult = AsyncDataStream.unorderedWait(input, function, 2);
+		DataStream<Integer> unorderedResult = AsyncDataStream.unorderedWait(
+			input,
+			function,
+			timeout,
+			TimeUnit.MILLISECONDS,
+			2);
 
 		// save result from unordered process
 		final MemorySinkFunction sinkFunction2 = new MemorySinkFunction(1);


[5/7] flink git commit: [FLINK-4391] Polish asynchronous I/O operations

Posted by tr...@apache.org.
[FLINK-4391] Polish asynchronous I/O operations

Polish AsyncFunction

Move AsyncCollectorBuffer to operators package

Rework AsyncWaitOperator and AsyncStreamElementQueue implementation

Rename AsyncCollectorQueue into StreamElementQueue

Reworked StreamingOperatorsITCase and RichAsyncFunctionTest

Refactor AsyncWaitOperatorTest

Add StreamElementQueueTests

Add EmitterTest case

Add comments


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ad603d59
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ad603d59
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ad603d59

Branch: refs/heads/master
Commit: ad603d59ec17a07adef995c2f1fd58fb8571a3d8
Parents: f528307
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Dec 14 17:37:39 2016 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Tue Dec 20 05:04:51 2016 +0100

----------------------------------------------------------------------
 .../org/apache/flink/util/ExceptionUtils.java   |  54 +-
 .../examples/async/AsyncIOExample.java          | 128 ++--
 .../api/functions/async/AsyncFunction.java      |  22 +-
 .../api/functions/async/RichAsyncFunction.java  | 156 ++---
 .../async/buffer/AbstractBufferEntry.java       |  78 ---
 .../async/buffer/AsyncCollectorBuffer.java      | 633 ------------------
 .../async/buffer/LatencyMarkerEntry.java        |  36 -
 .../async/buffer/StreamElementEntry.java        |  82 ---
 .../async/buffer/StreamRecordEntry.java         |  75 ---
 .../functions/async/buffer/WatermarkEntry.java  |  36 -
 .../async/collector/AsyncCollector.java         |  11 +-
 .../api/operators/TimestampedCollector.java     |   4 +
 .../api/operators/async/AsyncWaitOperator.java  | 324 ++++++---
 .../streaming/api/operators/async/Emitter.java  | 152 +++++
 .../api/operators/async/OperatorActions.java    |  34 +
 .../async/queue/AsyncCollectionResult.java      |  41 ++
 .../api/operators/async/queue/AsyncResult.java  |  59 ++
 .../async/queue/AsyncWatermarkResult.java       |  33 +
 .../async/queue/OrderedStreamElementQueue.java  | 229 +++++++
 .../async/queue/StreamElementQueue.java         |  96 +++
 .../async/queue/StreamElementQueueEntry.java    |  97 +++
 .../async/queue/StreamRecordQueueEntry.java     |  85 +++
 .../queue/UnorderedStreamElementQueue.java      | 304 +++++++++
 .../async/queue/WatermarkQueueEntry.java        |  47 ++
 .../streaming/runtime/tasks/OperatorChain.java  |   1 -
 .../streaming/runtime/tasks/StreamTask.java     |  10 +-
 .../functions/async/RichAsyncFunctionTest.java  | 283 +++++---
 .../async/AsyncCollectorBufferTest.java         | 656 -------------------
 .../operators/async/AsyncWaitOperatorTest.java  | 315 ++++-----
 .../api/operators/async/EmitterTest.java        | 193 ++++++
 .../queue/OrderedStreamElementQueueTest.java    | 124 ++++
 .../async/queue/StreamElementQueueTest.java     | 263 ++++++++
 .../queue/UnorderedStreamElementQueueTest.java  | 182 +++++
 .../operators/StreamSourceOperatorTest.java     |  30 +-
 .../flink/streaming/util/CollectorOutput.java   |  57 ++
 .../streaming/api/StreamingOperatorsITCase.java |  96 ++-
 36 files changed, 2852 insertions(+), 2174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
index 32bc1d2..d1357a8 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
@@ -26,10 +26,13 @@ package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 @Internal
 public final class ExceptionUtils {
 	public static final String STRINGIFIED_NULL_EXCEPTION = "(null)";
@@ -59,7 +62,56 @@ public final class ExceptionUtils {
 			return e.getClass().getName() + " (error while printing stack trace)";
 		}
 	}
-	
+
+	/**
+	 * Adds a new exception as a {@link Throwable#addSuppressed(Throwable) suppressed exception}
+	 * to a prior exception, or returns the new exception, if no prior exception exists.
+	 *
+	 * <pre>{@code
+	 *
+	 * public void closeAllThings() throws Exception {
+	 *     Exception ex = null;
+	 *     try {
+	 *         component.shutdown();
+	 *     } catch (Exception e) {
+	 *         ex = firstOrSuppressed(e, ex);
+	 *     }
+	 *     try {
+	 *         anotherComponent.stop();
+	 *     } catch (Exception e) {
+	 *         ex = firstOrSuppressed(e, ex);
+	 *     }
+	 *     try {
+	 *         lastComponent.shutdown();
+	 *     } catch (Exception e) {
+	 *         ex = firstOrSuppressed(e, ex);
+	 *     }
+	 *
+	 *     if (ex != null) {
+	 *         throw ex;
+	 *     }
+	 * }
+	 * }</pre>
+	 *
+	 * @param newException The newly occurred exception
+	 * @param previous     The previously occurred exception, possibly null.
+	 *
+	 * @return The new exception, if no previous exception exists, or the previous exception with the
+	 *         new exception in the list of suppressed exceptions.
+	 */
+	public static <T extends Throwable> T firstOrSuppressed(T newException, @Nullable T previous) {
+		checkNotNull(newException, "newException");
+
+		if (previous == null) {
+			return newException;
+		} else {
+			previous.addSuppressed(newException);
+			return previous;
+		}
+	}
+
+
+
 	/**
 	 * Throws the given {@code Throwable} in scenarios where the signatures do not allow you to
 	 * throw an arbitrary Throwable. Errors and RuntimeExceptions are thrown directly, other exceptions

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
index 96c7658..6dde537 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java
@@ -33,6 +33,8 @@ import org.apache.flink.streaming.api.functions.async.RichAsyncFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
 import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -43,10 +45,17 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
 /**
- * Example to illustrates how to use {@link org.apache.flink.streaming.api.functions.async.AsyncFunction}
+ * Example to illustrates how to use {@link AsyncFunction}
  */
 public class AsyncIOExample {
 
+	private static final Logger LOG = LoggerFactory.getLogger(AsyncIOExample.class);
+
+	private static final String EXACTLY_ONCE_MODE = "exactly_once";
+	private static final String EVENT_TIME = "EventTime";
+	private static final String INGESTION_TIME = "IngestionTime";
+	private static final String ORDERED = "ordered";
+
 	/**
 	 * A checkpointed source.
 	 */
@@ -103,8 +112,10 @@ public class AsyncIOExample {
 	 * async client.
 	 */
 	private static class SampleAsyncFunction extends RichAsyncFunction<Integer, String> {
-		transient static ExecutorService executorService;
-		transient static Random random;
+		private static final long serialVersionUID = 2098635244857937717L;
+
+		private static ExecutorService executorService;
+		private static Random random;
 
 		private int counter;
 
@@ -112,17 +123,17 @@ public class AsyncIOExample {
 		 * The result of multiplying sleepFactor with a random float is used to pause
 		 * the working thread in the thread pool, simulating a time consuming async operation.
 		 */
-		final long sleepFactor;
+		private final long sleepFactor;
 
 		/**
 		 * The ratio to generate an exception to simulate an async error. For example, the error
 		 * may be a TimeoutException while visiting HBase.
 		 */
-		final float failRatio;
+		private final float failRatio;
 
-		final long shutdownWaitTS;
+		private final long shutdownWaitTS;
 
-		public SampleAsyncFunction(long sleepFactor, float failRatio, long shutdownWaitTS) {
+		SampleAsyncFunction(long sleepFactor, float failRatio, long shutdownWaitTS) {
 			this.sleepFactor = sleepFactor;
 			this.failRatio = failRatio;
 			this.shutdownWaitTS = shutdownWaitTS;
@@ -155,7 +166,9 @@ public class AsyncIOExample {
 					executorService.shutdown();
 
 					try {
-						executorService.awaitTermination(shutdownWaitTS, TimeUnit.MILLISECONDS);
+						if (!executorService.awaitTermination(shutdownWaitTS, TimeUnit.MILLISECONDS)) {
+							executorService.shutdownNow();
+						}
 					} catch (InterruptedException e) {
 						executorService.shutdownNow();
 					}
@@ -169,14 +182,15 @@ public class AsyncIOExample {
 				@Override
 				public void run() {
 					// wait for while to simulate async operation here
-					int sleep = (int) (random.nextFloat() * sleepFactor);
+					long sleep = (long) (random.nextFloat() * sleepFactor);
 					try {
 						Thread.sleep(sleep);
-						List<String> ret = Collections.singletonList("key-" + (input % 10));
+
 						if (random.nextFloat() < failRatio) {
 							collector.collect(new Exception("wahahahaha..."));
 						} else {
-							collector.collect(ret);
+							collector.collect(
+								Collections.singletonList("key-" + (input % 10)));
 						}
 					} catch (InterruptedException e) {
 						collector.collect(new ArrayList<String>(0));
@@ -200,47 +214,71 @@ public class AsyncIOExample {
 		// obtain execution environment
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		printUsage();
-
 		// parse parameters
 		final ParameterTool params = ParameterTool.fromArgs(args);
 
-		// check the configuration for the job
-		final String statePath = params.getRequired("fsStatePath");
-		final String cpMode = params.get("checkpointMode", "exactly_once");
-		final int maxCount = params.getInt("maxCount", 100000);
-		final int sleepFactor = params.getInt("sleepFactor", 100);
-		final float failRatio = params.getFloat("failRatio", 0.001f);
-		final String mode = params.get("waitMode", "ordered");
-		final int taskNum =  params.getInt("waitOperatorParallelism", 1);
-		final String timeType = params.get("eventType", "EventTime");
-		final int shutdownWaitTS = params.getInt("shutdownWaitTS", 20000);
-
-		System.out.println("Job configuration\n"
-			+"\tFS state path="+statePath+"\n"
-			+"\tCheckpoint mode="+cpMode+"\n"
-			+"\tMax count of input from source="+maxCount+"\n"
-			+"\tSleep factor="+sleepFactor+"\n"
-			+"\tFail ratio="+failRatio+"\n"
-			+"\tWaiting mode="+mode+"\n"
-			+"\tParallelism for async wait operator="+taskNum+"\n"
-			+"\tEvent type="+timeType+"\n"
-			+"\tShutdown wait timestamp="+shutdownWaitTS);
-
-		// setup state and checkpoint mode
-		env.setStateBackend(new FsStateBackend(statePath));
-		if (cpMode.equals("exactly_once")) {
-			env.enableCheckpointing(1000, CheckpointingMode.EXACTLY_ONCE);
+		final String statePath;
+		final String cpMode;
+		final int maxCount;
+		final long sleepFactor;
+		final float failRatio;
+		final String mode;
+		final int taskNum;
+		final String timeType;
+		final long shutdownWaitTS;
+
+		try {
+			// check the configuration for the job
+			statePath = params.get("fsStatePath", null);
+			cpMode = params.get("checkpointMode", "exactly_once");
+			maxCount = params.getInt("maxCount", 100000);
+			sleepFactor = params.getLong("sleepFactor", 100);
+			failRatio = params.getFloat("failRatio", 0.001f);
+			mode = params.get("waitMode", "ordered");
+			taskNum = params.getInt("waitOperatorParallelism", 1);
+			timeType = params.get("eventType", "EventTime");
+			shutdownWaitTS = params.getLong("shutdownWaitTS", 20000);
+		} catch (Exception e) {
+			printUsage();
+
+			throw e;
+		}
+
+		StringBuilder configStringBuilder = new StringBuilder();
+
+		final String lineSeparator = System.getProperty("line.separator");
+
+		configStringBuilder
+			.append("Job configuration").append(lineSeparator)
+			.append("FS state path=").append(statePath).append(lineSeparator)
+			.append("Checkpoint mode=").append(cpMode).append(lineSeparator)
+			.append("Max count of input from source=").append(maxCount).append(lineSeparator)
+			.append("Sleep factor=").append(sleepFactor).append(lineSeparator)
+			.append("Fail ratio=").append(failRatio).append(lineSeparator)
+			.append("Waiting mode=").append(mode).append(lineSeparator)
+			.append("Parallelism for async wait operator=").append(taskNum).append(lineSeparator)
+			.append("Event type=").append(timeType).append(lineSeparator)
+			.append("Shutdown wait timestamp=").append(shutdownWaitTS);
+
+		LOG.info(configStringBuilder.toString());
+
+		if (statePath != null) {
+			// setup state and checkpoint mode
+			env.setStateBackend(new FsStateBackend(statePath));
+		}
+
+		if (EXACTLY_ONCE_MODE.equals(cpMode)) {
+			env.enableCheckpointing(1000L, CheckpointingMode.EXACTLY_ONCE);
 		}
 		else {
-			env.enableCheckpointing(1000, CheckpointingMode.AT_LEAST_ONCE);
+			env.enableCheckpointing(1000L, CheckpointingMode.AT_LEAST_ONCE);
 		}
 
 		// enable watermark or not
-		if (timeType.equals("EventTime")) {
+		if (EVENT_TIME.equals(timeType)) {
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		}
-		else if (timeType.equals("IngestionTime")) {
+		else if (INGESTION_TIME.equals(timeType)) {
 			env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
 		}
 
@@ -253,7 +291,7 @@ public class AsyncIOExample {
 
 		// add async operator to streaming job
 		DataStream<String> result;
-		if (mode.equals("ordered")) {
+		if (ORDERED.equals(mode)) {
 			result = AsyncDataStream.orderedWait(inputStream, function, 20).setParallelism(taskNum);
 		}
 		else {
@@ -262,6 +300,8 @@ public class AsyncIOExample {
 
 		// add a reduce to get the sum of each keys.
 		result.flatMap(new FlatMapFunction<String, Tuple2<String, Integer>>() {
+			private static final long serialVersionUID = -938116068682344455L;
+
 			@Override
 			public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception {
 				out.collect(new Tuple2<>(value, 1));
@@ -269,7 +309,7 @@ public class AsyncIOExample {
 		}).keyBy(0).sum(1).print();
 
 		// execute the program
-		env.execute("Async I/O Example");
+		env.execute("Async IO Example");
 	}
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
index b5b7d6f..4de2db1 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
@@ -29,14 +29,15 @@ import java.io.Serializable;
  * <p>
  * For each #asyncInvoke, an async io operation can be triggered, and once it has been done,
  * the result can be collected by calling {@link AsyncCollector#collect}. For each async
- * operations, their contexts are buffered in the operator immediately after invoking
- * #asyncInvoke, leading to no blocking for each stream input as long as internal buffer is not full.
+ * operation, its context is stored in the operator immediately after invoking
+ * #asyncInvoke, avoiding blocking for each stream input as long as the internal buffer is not full.
  * <p>
- * {@link AsyncCollector} can be passed into callbacks or futures provided by async client to
- * fetch result data. Any error can also be propagate to the operator by {@link AsyncCollector#collect(Throwable)}.
+ * {@link AsyncCollector} can be passed into callbacks or futures to collect the result data.
+ * An error can also be propagate to the async IO operator by
+ * {@link AsyncCollector#collect(Throwable)}.
  *
  * <p>
- * Typical usage for callback:
+ * Callback example usage:
  * <pre>{@code
  * public class HBaseAsyncFunc implements AsyncFunction<String, String> {
  *   @Override
@@ -46,11 +47,10 @@ import java.io.Serializable;
  *     hbase.asyncGet(get, cb);
  *   }
  * }
- * }
  * </pre>
  *
  * <p>
- * Typical usage for {@link com.google.common.util.concurrent.ListenableFuture}
+ * Future example usage:
  * <pre>{@code
  * public class HBaseAsyncFunc implements AsyncFunction<String, String> {
  *   @Override
@@ -68,7 +68,6 @@ import java.io.Serializable;
  *     });
  *   }
  * }
- * }
  * </pre>
  *
  * @param <IN> The type of the input elements.
@@ -80,9 +79,10 @@ public interface AsyncFunction<IN, OUT> extends Function, Serializable {
 	/**
 	 * Trigger async operation for each stream input.
 	 *
-	 * @param input Stream Input
-	 * @param collector AsyncCollector
-	 * @exception Exception will make task fail and trigger fail-over process.
+	 * @param input element coming from an upstream task
+	 * @param collector to collect the result data
+	 * @exception Exception in case of a user code error. An exception will make the task fail and
+	 * trigger fail-over process.
 	 */
 	void asyncInvoke(IN input, AsyncCollector<OUT> collector) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java
index f6d3d31..232206c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java
@@ -41,6 +41,7 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
 import org.apache.flink.types.Value;
+import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
 import java.util.List;
@@ -48,88 +49,55 @@ import java.util.Map;
 
 /**
  * Rich variant of the {@link AsyncFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RuntimeContext} and provides setup and teardown methods:
  * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
  * {@link RichFunction#close()}.
  *
  * <p>
- * State related apis in {@link RuntimeContext} are not supported yet because the key may get changed
- * while accessing states in the working thread.
+ * State related apis in {@link RuntimeContext} are not supported yet because the key may get
+ * changed while accessing states in the working thread.
  * <p>
- * {@link IterationRuntimeContext#getIterationAggregator(String)} is not supported since the aggregator
- * may be modified by multiple threads.
+ * {@link IterationRuntimeContext#getIterationAggregator(String)} is not supported since the
+ * aggregator may be modified by multiple threads.
  *
  * @param <IN> The type of the input elements.
  * @param <OUT> The type of the returned elements.
  */
-
 @PublicEvolving
-public abstract class RichAsyncFunction<IN, OUT> extends AbstractRichFunction
-	implements AsyncFunction<IN, OUT> {
+public abstract class RichAsyncFunction<IN, OUT> extends AbstractRichFunction implements AsyncFunction<IN, OUT> {
 
-	private transient RuntimeContext runtimeContext;
+	private static final long serialVersionUID = 3858030061138121840L;
 
 	@Override
-	public void setRuntimeContext(RuntimeContext t) {
-		super.setRuntimeContext(t);
+	public void setRuntimeContext(RuntimeContext runtimeContext) {
+		Preconditions.checkNotNull(runtimeContext);
 
-		if (t != null) {
-			runtimeContext = new RichAsyncFunctionRuntimeContext(t);
+		if (runtimeContext instanceof IterationRuntimeContext) {
+			super.setRuntimeContext(
+				new RichAsyncFunctionIterationRuntimeContext(
+					(IterationRuntimeContext) runtimeContext));
+		} else {
+			super.setRuntimeContext(new RichAsyncFunctionRuntimeContext(runtimeContext));
 		}
 	}
 
 	@Override
 	public abstract void asyncInvoke(IN input, AsyncCollector<OUT> collector) throws Exception;
 
-	@Override
-	public RuntimeContext getRuntimeContext() {
-		if (this.runtimeContext != null) {
-			return runtimeContext;
-		} else {
-			throw new IllegalStateException("The runtime context has not been initialized.");
-		}
-	}
-
-	@Override
-	public IterationRuntimeContext getIterationRuntimeContext() {
-		if (this.runtimeContext != null) {
-			return (IterationRuntimeContext) runtimeContext;
-		} else {
-			throw new IllegalStateException("The runtime context has not been initialized.");
-		}
-	}
+	// -----------------------------------------------------------------------------------------
+	// Wrapper classes
+	// -----------------------------------------------------------------------------------------
 
 	/**
-	 * A wrapper class to delegate {@link RuntimeContext}. State related apis are disabled.
+	 * A wrapper class for async function's {@link RuntimeContext}. The async function runtime
+	 * context only supports basic operations which are thread safe. Consequently, state access,
+	 * accumulators, broadcast variables and the distributed cache are disabled.
 	 */
-	private class RichAsyncFunctionRuntimeContext implements IterationRuntimeContext {
-		private RuntimeContext runtimeContext;
-
-		public RichAsyncFunctionRuntimeContext(RuntimeContext context) {
-			runtimeContext = context;
-		}
-
-		private IterationRuntimeContext getIterationRuntineContext() {
-			if (this.runtimeContext instanceof IterationRuntimeContext) {
-				return (IterationRuntimeContext) this.runtimeContext;
-			} else {
-				throw new IllegalStateException("This stub is not part of an iteration step function.");
-			}
-		}
-
-		@Override
-		public int getSuperstepNumber() {
-			return getIterationRuntineContext().getSuperstepNumber();
-		}
-
-		@Override
-		public <T extends Aggregator<?>> T getIterationAggregator(String name) {
-			throw new UnsupportedOperationException("Get iteration aggregator is not supported in rich async function");
-		}
+	private static class RichAsyncFunctionRuntimeContext implements RuntimeContext {
+		private final RuntimeContext runtimeContext;
 
-		@Override
-		public <T extends Value> T getPreviousIterationAggregate(String name) {
-			return getIterationRuntineContext().getPreviousIterationAggregate(name);
+		RichAsyncFunctionRuntimeContext(RuntimeContext context) {
+			runtimeContext = Preconditions.checkNotNull(context);
 		}
 
 		@Override
@@ -172,74 +140,108 @@ public abstract class RichAsyncFunction<IN, OUT> extends AbstractRichFunction
 			return runtimeContext.getUserCodeClassLoader();
 		}
 
+		// -----------------------------------------------------------------------------------
+		// Unsupported operations
+		// -----------------------------------------------------------------------------------
+
+		@Override
+		public DistributedCache getDistributedCache() {
+			throw new UnsupportedOperationException("Distributed cache is not supported in rich async functions.");
+		}
+
+		@Override
+		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("State is not supported in rich async functions.");
+		}
+
+		@Override
+		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("State is not supported in rich async functions.");
+		}
+
+		@Override
+		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("State is not supported in rich async functions.");
+		}
+
 		@Override
 		public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {
-			runtimeContext.addAccumulator(name, accumulator);
+			throw new UnsupportedOperationException("Accumulators are not supported in rich async functions.");
 		}
 
 		@Override
 		public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
-			return runtimeContext.getAccumulator(name);
+			throw new UnsupportedOperationException("Accumulators are not supported in rich async functions.");
 		}
 
 		@Override
 		public Map<String, Accumulator<?, ?>> getAllAccumulators() {
-			return runtimeContext.getAllAccumulators();
+			throw new UnsupportedOperationException("Accumulators are not supported in rich async functions.");
 		}
 
 		@Override
 		public IntCounter getIntCounter(String name) {
-			return runtimeContext.getIntCounter(name);
+			throw new UnsupportedOperationException("Int counters are not supported in rich async functions.");
 		}
 
 		@Override
 		public LongCounter getLongCounter(String name) {
-			return runtimeContext.getLongCounter(name);
+			throw new UnsupportedOperationException("Long counters are not supported in rich async functions.");
 		}
 
 		@Override
 		public DoubleCounter getDoubleCounter(String name) {
-			return runtimeContext.getDoubleCounter(name);
+			throw new UnsupportedOperationException("Long counters are not supported in rich async functions.");
 		}
 
 		@Override
 		public Histogram getHistogram(String name) {
-			return runtimeContext.getHistogram(name);
+			throw new UnsupportedOperationException("Histograms are not supported in rich async functions.");
 		}
 
 		@Override
 		public boolean hasBroadcastVariable(String name) {
-			return runtimeContext.hasBroadcastVariable(name);
+			throw new UnsupportedOperationException("Broadcast variables are not supported in rich async functions.");
 		}
 
 		@Override
 		public <RT> List<RT> getBroadcastVariable(String name) {
-			return runtimeContext.getBroadcastVariable(name);
+			throw new UnsupportedOperationException("Broadcast variables are not supported in rich async functions.");
 		}
 
 		@Override
 		public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
-			return runtimeContext.getBroadcastVariableWithInitializer(name, initializer);
+			throw new UnsupportedOperationException("Broadcast variables are not supported in rich async functions.");
 		}
+	}
 
-		@Override
-		public DistributedCache getDistributedCache() {
-			return runtimeContext.getDistributedCache();
+	private static class RichAsyncFunctionIterationRuntimeContext extends RichAsyncFunctionRuntimeContext implements IterationRuntimeContext {
+
+		private final IterationRuntimeContext iterationRuntimeContext;
+
+		RichAsyncFunctionIterationRuntimeContext(IterationRuntimeContext iterationRuntimeContext) {
+			super(iterationRuntimeContext);
+
+			this.iterationRuntimeContext = Preconditions.checkNotNull(iterationRuntimeContext);
 		}
 
 		@Override
-		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
-			throw new UnsupportedOperationException("State is not supported in rich async function");
+		public int getSuperstepNumber() {
+			return iterationRuntimeContext.getSuperstepNumber();
 		}
 
+		// -----------------------------------------------------------------------------------
+		// Unsupported operations
+		// -----------------------------------------------------------------------------------
+
 		@Override
-		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
-			throw new UnsupportedOperationException("State is not supported in rich async function");
+		public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+			throw new UnsupportedOperationException("Iteration aggregators are not supported in rich async functions.");
 		}
 
 		@Override
-		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
-			throw new UnsupportedOperationException("State is not supported in rich async function");
+		public <T extends Value> T getPreviousIterationAggregate(String name) {
+			throw new UnsupportedOperationException("Iteration aggregators are not supported in rich async functions.");
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java
deleted file mode 100644
index 29643fd..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AbstractBufferEntry.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.async.buffer;
-
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * Abstract implementation for {@link StreamElementEntry}
- *
- * @param <OUT> Output type.
- */
-public abstract class AbstractBufferEntry<OUT> implements StreamElementEntry<OUT> {
-	private final StreamElement streamElement;
-
-	protected AbstractBufferEntry(StreamElement element) {
-		this.streamElement = Preconditions.checkNotNull(element, "Reference to StreamElement should not be null");
-	}
-
-	@Override
-	public List<OUT> getResult() throws IOException {
-		throw new UnsupportedOperationException("It is only available for StreamRecordEntry");
-	}
-
-	@Override
-	public void markDone() {
-		throw new UnsupportedOperationException("It is only available for StreamRecordEntry");
-	}
-
-	@Override
-	public boolean isDone() {
-		throw new UnsupportedOperationException("It must be overriden by the concrete entry");
-	}
-
-	@Override
-	public boolean isStreamRecord() {
-		return streamElement.isRecord();
-	}
-
-	@Override
-	public boolean isWatermark() {
-		return streamElement.isWatermark();
-	}
-
-	@Override
-	public boolean isLatencyMarker() {
-		return streamElement.isLatencyMarker();
-	}
-
-	@Override
-	public StreamElement getStreamElement() {
-		return streamElement;
-	}
-
-	@Override
-	public String toString() {
-		return "StreamElementEntry for @" + streamElement;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java
deleted file mode 100644
index ee176d9..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/AsyncCollectorBuffer.java
+++ /dev/null
@@ -1,633 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.async.buffer;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.streaming.api.datastream.AsyncDataStream;
-import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
-import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-
-/**
- * AsyncCollectorBuffer will hold all {@link AsyncCollector} in its internal buffer,
- * and emit results from {@link AsyncCollector} to the next operators following it by
- * calling {@link Output#collect(Object)}
- */
-@Internal
-public class AsyncCollectorBuffer<IN, OUT> {
-
-	/**
-	 * The logger.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(AsyncCollectorBuffer.class);
-
-	/**
-	 * Max number of {@link AsyncCollector} in the buffer.
-	 */
-	private final int bufferSize;
-
-	private final AsyncDataStream.OutputMode mode;
-
-	private final AsyncWaitOperator<IN, OUT> operator;
-
-	/**
-	 * Keep all {@link StreamElementEntry}
-	 */
-	private final Set<StreamElementEntry<OUT>> queue = new LinkedHashSet<>();
-
-	/**
-	 * Keep all {@link StreamElementEntry} to their corresponding {@link Watermark} or {@link LatencyMarker}
-	 * If the inputs are: SR1, SR2, WM1, SR3, SR4. Then SR1 and SR2 belong to WM1, and
-	 * SR3 and SR4 will be kept in {@link #lonelyEntries}
-	 */
-	private final Map<StreamElementEntry<OUT>, StreamElement> entriesToMarkers = new HashMap<>();
-
-	private final List<StreamElementEntry<OUT>> lonelyEntries = new LinkedList<>();
-
-	/**
-	 * Keep finished AsyncCollector belonging to the oldest Watermark or LatencyMarker in UNORDERED mode.
-	 */
-	private final Map<StreamElement, Set<StreamElementEntry<OUT>>> markerToFinishedEntries = new LinkedHashMap<>();
-	private Set<StreamElementEntry<OUT>>lonelyFinishedEntries = new HashSet<>();
-
-	/**
-	 * For the AsyncWaitOperator chained with StreamSource, the checkpoint thread may get the
-	 * {@link org.apache.flink.streaming.runtime.tasks.StreamTask#lock} while {@link AsyncCollectorBuffer#queue}
-	 * is full since main thread waits on this lock. The StreamElement in
-	 * {@link AsyncWaitOperator#processElement(StreamRecord)} should be treated as a part of all StreamElements
-	 * in its queue. It will be kept in the operator state while snapshotting.
-	 */
-	private StreamElement extraStreamElement;
-
-	/**
-	 * {@link TimestampedCollector} and {@link Output} to collect results and watermarks.
-	 */
-	private final Output<StreamRecord<OUT>> output;
-	private final TimestampedCollector<OUT> timestampedCollector;
-
-	/**
-	 * Checkpoint lock from {@link org.apache.flink.streaming.runtime.tasks.StreamTask#lock}
-	 */
-	private final Object lock;
-
-	private final Emitter emitter;
-	private final Thread emitThread;
-
-	/**
-	 * Exception from async operation or internal error
-	 */
-	private Exception error;
-
-	/**
-	 * Flag telling Emitter thread to work or not.
-	 */
-	private volatile boolean workwork = false;
-
-	public AsyncCollectorBuffer(
-			int bufferSize,
-			AsyncDataStream.OutputMode mode,
-			Output<StreamRecord<OUT>> output,
-			TimestampedCollector<OUT> collector,
-			Object lock,
-			AsyncWaitOperator operator) {
-		Preconditions.checkArgument(bufferSize > 0, "Future buffer size should be greater than 0.");
-
-		this.bufferSize = bufferSize;
-
-		this.mode = Preconditions.checkNotNull(mode, "Processing mode should not be NULL.");
-		this.output = Preconditions.checkNotNull(output, "Output should not be NULL.");
-		this.timestampedCollector = Preconditions.checkNotNull(collector, "TimestampedCollector should not be NULL.");
-		this.operator = Preconditions.checkNotNull(operator, "Reference to AsyncWaitOperator should not be NULL.");
-		this.lock = Preconditions.checkNotNull(lock, "Checkpoint lock should not be NULL.");
-
-		this.emitter = new Emitter();
-		this.emitThread = new Thread(emitter);
-		this.emitThread.setDaemon(true);
-	}
-
-	/**
-	 * Add an {@link StreamRecord} into the buffer. A new {@link AsyncCollector} will be created and returned
-	 * corresponding to the input StreamRecord.
-	 * <p>
-	 * If buffer is full, caller will wait until a new space is available.
-	 *
-	 * @param record StreamRecord
-	 * @return An AsyncCollector
-	 * @throws Exception Exception from AsyncCollector.
-	 */
-	public AsyncCollector<OUT> addStreamRecord(StreamRecord<IN> record) throws Exception {
-		assert(Thread.holdsLock(lock));
-
-		while (queue.size() >= bufferSize) {
-			// hold the input StreamRecord until it is placed in the buffer
-			extraStreamElement = record;
-
-			lock.wait();
-		}
-
-		if (error != null) {
-			throw error;
-		}
-
-		StreamElementEntry<OUT> entry = new StreamRecordEntry<>(record, this);
-
-		queue.add(entry);
-
-		if (mode == AsyncDataStream.OutputMode.UNORDERED) {
-			lonelyEntries.add(entry);
-		}
-
-		extraStreamElement = null;
-
-		return (AsyncCollector<OUT>)entry;
-	}
-
-	/**
-	 * Add a {@link Watermark} into buffer.
-	 * <p>
-	 * If queue is full, caller will wait here.
-	 *
-	 * @param watermark Watermark
-	 * @throws Exception Exception from AsyncCollector.
-	 */
-	public void addWatermark(Watermark watermark) throws Exception {
-		processMark(new WatermarkEntry<OUT>(watermark));
-	}
-
-	/**
-	 * Add a {@link LatencyMarker} into buffer.
-	 * <p>
-	 * If queue is full, caller will wait here.
-	 *
-	 * @param latencyMarker LatencyMarker
-	 * @throws Exception Exception from AsyncCollector.
-	 */
-	public void addLatencyMarker(LatencyMarker latencyMarker) throws Exception {
-		processMark(new LatencyMarkerEntry<OUT>(latencyMarker));
-	}
-
-	/**
-	 * Notify the emitter thread and main thread that an AsyncCollector has completed.
-	 *
-	 * @param entry Completed AsyncCollector
-	 */
-	public void markCollectorCompleted(StreamElementEntry<OUT> entry) {
-		synchronized (lock) {
-			entry.markDone();
-
-			if (mode == AsyncDataStream.OutputMode.UNORDERED) {
-				StreamElement marker = entriesToMarkers.get(entry);
-
-				if (marker != null) {
-					markerToFinishedEntries.get(marker).add(entry);
-				}
-				else {
-					lonelyFinishedEntries.add(entry);
-				}
-			}
-
-			// if workwork is true, it is not necessary to check it again
-			if (!workwork && shouldNotifyEmitterThread(entry)) {
-				workwork = true;
-
-				lock.notifyAll();
-			}
-		}
-	}
-
-	/**
-	 * Caller will wait here if buffer is not empty, meaning that not all async i/o tasks have returned yet.
-	 *
-	 * @throws Exception IOException from AsyncCollector.
-	 */
-	public void waitEmpty() throws Exception {
-		assert(Thread.holdsLock(lock));
-
-		while (queue.size() != 0) {
-			if (error != null) {
-				throw error;
-			}
-
-			lock.wait();
-		}
-	}
-
-	public void startEmitterThread() {
-		emitThread.start();
-	}
-
-	public void stopEmitterThread() {
-		emitter.stop();
-
-		emitThread.interrupt();
-
-		while (emitThread.isAlive()) {
-			// temporarily release the lock first, since caller of this method may also hold the lock.
-			if (Thread.holdsLock(lock)) {
-				try {
-					lock.wait(1000);
-				}
-				catch (InterruptedException e) {
-					// do nothing
-				}
-			}
-
-			try {
-				emitThread.join(10000);
-			} catch (InterruptedException e) {
-				// do nothing
-			}
-
-			// get the stack trace
-			StringBuilder sb = new StringBuilder();
-			StackTraceElement[] stack = emitThread.getStackTrace();
-
-			for (StackTraceElement e : stack) {
-				sb.append(e).append('\n');
-			}
-
-			LOG.warn("Emitter thread blocks due to {}", sb.toString());
-
-			emitThread.interrupt();
-		}
-	}
-
-	/**
-	 * Get all StreamElements in the AsyncCollector queue.
-	 * <p>
-	 * Emitter Thread can not output records and will wait for a while due to checkpoiting procedure
-	 * holding the checkpoint lock.
-	 *
-	 * @return An {@link Iterator} to the StreamElements in the buffer, including the extra one.
-	 */
-	public Iterator<StreamElement> getStreamElementsInBuffer() {
-		final Iterator<StreamElementEntry<OUT>> iterator = queue.iterator();
-		final StreamElement extra = extraStreamElement;
-
-		return new Iterator<StreamElement>() {
-			boolean shouldSendExtraElement = (extra != null);
-
-			@Override
-			public boolean hasNext() {
-				return iterator.hasNext() || shouldSendExtraElement;
-			}
-
-			@Override
-			public StreamElement next() {
-				if (!hasNext()) {
-					throw new NoSuchElementException();
-				}
-
-				if (iterator.hasNext()) {
-					return iterator.next().getStreamElement();
-				}
-				else {
-					shouldSendExtraElement = false;
-
-					return extra;
-				}
-			}
-
-			@Override
-			public void remove() {
-				throw new UnsupportedOperationException("remove");
-			}
-		};
-	}
-
-	private void processMark(StreamElementEntry<OUT> entry) throws Exception {
-		assert(Thread.holdsLock(lock));
-
-		StreamElement mark = entry.getStreamElement();
-
-		while (queue.size() >= bufferSize) {
-			// hold the input StreamRecord until it is placed in the buffer
-			extraStreamElement = mark;
-
-			lock.wait();
-		}
-
-		if (error != null) {
-			throw error;
-		}
-
-		queue.add(entry);
-
-		if (mode == AsyncDataStream.OutputMode.UNORDERED) {
-			// update AsyncCollector to Watermark / LatencyMarker map
-			for (StreamElementEntry<OUT> e : lonelyEntries) {
-				entriesToMarkers.put(e, mark);
-			}
-
-			lonelyEntries.clear();
-
-			// update Watermark / LatencyMarker to finished AsyncCollector map
-			markerToFinishedEntries.put(mark, lonelyFinishedEntries);
-
-			lonelyFinishedEntries = new HashSet<>();
-		}
-
-		extraStreamElement = null;
-
-		// notify Emitter thread if the head of buffer is Watermark or LatencyMarker
-		// this is for the case when LatencyMarkers keep coming but there is no StreamRecords.
-		StreamElementEntry<OUT> element = queue.iterator().next();
-
-		if (element.isLatencyMarker() || element.isWatermark()) {
-			workwork = true;
-
-			lock.notifyAll();
-		}
-	}
-
-	private boolean shouldNotifyEmitterThread(StreamElementEntry<OUT> entry) {
-
-		switch (mode) {
-
-			case ORDERED:
-				Iterator<StreamElementEntry<OUT>> queueIterator = queue.iterator();
-
-				// get to work as long as the first AsyncCollect is done.
-				return queueIterator.hasNext() && (queueIterator.next().isDone());
-
-			case UNORDERED:
-				Iterator<Map.Entry<StreamElement, Set<StreamElementEntry<OUT>>>> iteratorMarker =
-						markerToFinishedEntries.entrySet().iterator();
-
-				// get to work only the finished AsyncCollector belongs to the oldest Watermark or LatencyMarker
-				// or no Watermark / LatencyMarker is in the buffer yet.
-				return iteratorMarker.hasNext() ? iteratorMarker.next().getValue().contains(entry)
-						: lonelyFinishedEntries.contains(entry);
-
-			default:
-				// this case should never happen
-				return false;
-		}
-	}
-
-	@VisibleForTesting
-	public Set<StreamElementEntry<OUT>> getQueue() {
-		return queue;
-	}
-
-	@VisibleForTesting
-	public void setExtraStreamElement(StreamElement element) {
-		extraStreamElement = element;
-	}
-
-	/**
-	 * A working thread to output results from {@link AsyncCollector} to the next operator.
-	 */
-	private class Emitter implements Runnable {
-		private volatile boolean running = true;
-
-		private void output(StreamElementEntry<OUT> entry) throws Exception {
-
-			StreamElement element = entry.getStreamElement();
-
-			if (element == null) {
-				throw new Exception("StreamElement in the buffer entry should not be null");
-			}
-
-			if (entry.isStreamRecord()) {
-				List<OUT> result = entry.getResult();
-
-				if (result == null) {
-					throw new Exception("Result for stream record " + element + " is null");
-				}
-
-				// update the timestamp for the collector
-				timestampedCollector.setTimestamp(element.asRecord());
-
-				for (OUT val : result) {
-					timestampedCollector.collect(val);
-				}
-			}
-			else if (entry.isWatermark()) {
-				output.emitWatermark(element.asWatermark());
-			}
-			else if (entry.isLatencyMarker()) {
-				operator.sendLatencyMarker(element.asLatencyMarker());
-			}
-			else {
-				throw new IOException("Unknown input record: " + element);
-			}
-		}
-
-		/**
-		 * Emit results from the finished head collector and its following finished ones.
-		 *
-		 * <p>NOTE: Since {@link #output(StreamElementEntry)} may be blocked if operator chain chained with
-		 * another {@link AsyncWaitOperator} and its buffer is full, we can not use an {@link Iterator} to
-		 * go through {@link #queue} because ConcurrentModificationException may be thrown while we remove
-		 * element in the queue by calling {@link Iterator#remove()}.
-		 *
-		 * <p>Example: Assume operator chain like this: async-wait-operator1(awo1) -> async-wait-operator2(awo2).
-		 * The buffer for awo1 is full so the main thread is blocked there.
-		 * The {@link Emitter} thread, named emitter1, in awo1 is outputting
-		 * data to awo2. Assume that 2 elements have been emitted and the buffer in awo1 has two vacancies. While
-		 * outputting the third one, the buffer in awo2 is full, so emitter1 will wait for a moment. If we use
-		 * {@link Iterator}, it is just before calling {@link Iterator#remove()}. Once the {@link #lock} is released
-		 * and luckily enough, the main thread get the lock. It will modify {@link #queue}, causing
-		 * ConcurrentModificationException once emitter1 runs to {@link Iterator#remove()}.
-		 *
-		 */
-		private void orderedProcess() throws Exception {
-			StreamElementEntry<OUT> entry;
-
-			while (queue.size() > 0 && (entry = queue.iterator().next()).isDone()) {
-				output(entry);
-
-				queue.remove(entry);
-			}
-		}
-
-		/**
-		 * Emit results for each finished collector. Try to emit results prior to the oldest watermark
-		 * in the buffer.
-		 * <p>
-		 * For example, assume the sequence of input StreamElements is:
-		 * Entry(ac1, record1) -> Entry(ac2, record2) -> Entry(ac3, watermark1) -> Entry(ac4, record3).
-		 * and both of ac2 and ac3 have finished. For unordered-mode, ac1 and ac2 are prior to watermark1,
-		 * so ac2 will be emitted. Since ac1 is not ready yet, ac3 have to wait until ac1 is done.
-		 */
-		private void unorderedProcess() throws Exception {
-			// try to emit finished AsyncCollectors in markerToFinishedEntries
-			if (markerToFinishedEntries.size() != 0) {
-				while (markerToFinishedEntries.size() != 0) {
-					Map.Entry<StreamElement, Set<StreamElementEntry<OUT>>> finishedStreamElementEntry =
-							markerToFinishedEntries.entrySet().iterator().next();
-
-					Set<StreamElementEntry<OUT>> finishedElementSet = finishedStreamElementEntry.getValue();
-
-					// While outputting results to the next operator, output may release lock if the following operator
-					// in the chain is another AsyncWaitOperator. During this period, there may be some
-					// finished StreamElementEntry coming into the finishedElementSet, and we should
-					// output all finished elements after re-acquiring the lock.
-					while (finishedElementSet.size() != 0) {
-						StreamElementEntry<OUT> finishedEntry = finishedElementSet.iterator().next();
-
-						output(finishedEntry);
-
-						queue.remove(finishedEntry);
-
-						entriesToMarkers.remove(finishedEntry);
-
-						finishedElementSet.remove(finishedEntry);
-					}
-
-					finishedStreamElementEntry.getValue().clear();
-
-
-					// if all StreamElements belonging to current Watermark / LatencyMarker have been emitted,
-					// emit current Watermark / LatencyMarker
-
-					if (queue.size() == 0) {
-						if (markerToFinishedEntries.size() != 0 || entriesToMarkers.size() != 0
-								|| lonelyEntries.size() != 0 || lonelyFinishedEntries.size() != 0) {
-							throw new IOException("Inner data info is not consistent.");
-						}
-					}
-					else {
-						// check the head AsyncCollector whether it is a Watermark or LatencyMarker.
-						StreamElementEntry<OUT> queueEntry = queue.iterator().next();
-
-						if (!queueEntry.isStreamRecord()) {
-							if (finishedStreamElementEntry.getKey() != queueEntry.getStreamElement()) {
-								throw new IOException("Watermark / LatencyMarker from finished collector map "
-									+ "and input buffer are not the same.");
-							}
-
-							output(queueEntry);
-
-							queue.remove(queueEntry);
-
-							// remove useless data in markerToFinishedEntries
-							markerToFinishedEntries.remove(finishedStreamElementEntry.getKey());
-						}
-						else {
-							break;
-						}
-					}
-				}
-			}
-
-			if (markerToFinishedEntries.size() == 0) {
-				// health check
-				if (entriesToMarkers.size() != 0) {
-					throw new IOException("Entries to marker map should be zero");
-				}
-
-				// no Watermark or LatencyMarker in the buffer yet, emit results in lonelyFinishedEntries
-				while (lonelyFinishedEntries.size() != 0) {
-					StreamElementEntry<OUT> entry = lonelyFinishedEntries.iterator().next();
-
-					output(entry);
-
-					queue.remove(entry);
-
-					lonelyEntries.remove(entry);
-
-					lonelyFinishedEntries.remove(entry);
-				}
-			}
-		}
-
-		private void processFinishedAsyncCollector() throws Exception {
-			if (mode == AsyncDataStream.OutputMode.ORDERED) {
-				orderedProcess();
-			} else {
-				unorderedProcess();
-			}
-		}
-
-		private void clearAndNotify() {
-			// clear all data
-			queue.clear();
-			entriesToMarkers.clear();
-			markerToFinishedEntries.clear();
-			lonelyEntries.clear();
-
-			running = false;
-
-			lock.notifyAll();
-		}
-
-		@Override
-		public void run() {
-			while (running) {
-				synchronized (lock) {
-
-					try {
-						while (!workwork) {
-							lock.wait();
-						}
-
-						processFinishedAsyncCollector();
-
-						lock.notifyAll();
-
-						workwork = false;
-					}
-					catch (InterruptedException e) {
-						// The source of InterruptedException is from:
-						//   1. lock.wait() statement in Emit
-						//   2. collector waiting for vacant buffer
-						// The action for this exception should try to clear all held data and
-						// exit Emit thread.
-
-						clearAndNotify();
-					}
-					catch (Exception e) {
-						// For exceptions, not InterruptedException, it should be propagated
-						// to main thread.
-						error = e;
-
-						clearAndNotify();
-					}
-				}
-			}
-		}
-
-		public void stop() {
-			running = false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java
deleted file mode 100644
index 1705c2d..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/LatencyMarkerEntry.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.async.buffer;
-
-import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
-
-/**
- * {@link AsyncCollectorBuffer} entry for {@link LatencyMarker}
- *
- */
-public class LatencyMarkerEntry<OUT> extends AbstractBufferEntry<OUT> {
-	public LatencyMarkerEntry(LatencyMarker marker) {
-		super(marker);
-	}
-
-	@Override
-	public boolean isDone() {
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java
deleted file mode 100644
index de7f606..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamElementEntry.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.async.buffer;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * The base class for entries in the {@link AsyncCollectorBuffer}
- *
- * @param <OUT> Output data type
- */
-
-@Internal
-public interface StreamElementEntry<OUT>  {
-	/**
-	 * Get result. Throw IOException while encountering an error.
-	 *
-	 * @return A List of result.
-	 * @throws IOException IOException wrapping errors from user codes.
-	 */
-	List<OUT> getResult() throws IOException;
-
-	/**
-	 * Set the internal flag, marking the async operator has finished.
-	 */
-	void markDone();
-
-	/**
-	 * Get the flag indicating the async operator has finished or not.
-	 *
-	 * @return True for finished async operator.
-	 */
-	boolean isDone();
-
-	/**
-	 * Check inner element is StreamRecord or not.
-	 *
-	 * @return True if element is StreamRecord.
-	 */
-	boolean isStreamRecord();
-
-	/**
-	 * Check inner element is Watermark or not.
-	 *
-	 * @return True if element is Watermark.
-	 */
-	boolean isWatermark();
-
-	/**
-	 * Check inner element is LatencyMarker or not.
-	 *
-	 * @return True if element is LatencyMarker.
-	 */
-	boolean isLatencyMarker();
-
-	/**
-	 * Get inner stream element.
-	 *
-	 * @return Inner {@link StreamElement}.
-	 */
-	StreamElement getStreamElement();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java
deleted file mode 100644
index fb0dc3b..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/StreamRecordEntry.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.async.buffer;
-
-import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * {@link AsyncCollectorBuffer} entry for {@link StreamRecord}
- *
- * @param <IN> Input data type
- * @param <OUT> Output data type
- */
-public class StreamRecordEntry<IN, OUT> extends AbstractBufferEntry<OUT> implements AsyncCollector<OUT> {
-	private List<OUT> result;
-	private Throwable error;
-
-	private boolean isDone = false;
-
-	private final AsyncCollectorBuffer<IN, OUT> buffer;
-
-	public StreamRecordEntry(StreamRecord<IN> element, AsyncCollectorBuffer<IN, OUT> buffer) {
-		super(element);
-		this.buffer = Preconditions.checkNotNull(buffer, "Reference to AsyncCollectorBuffer should not be null");
-	}
-
-	@Override
-	public void collect(List<OUT> result)  {
-		this.result = result;
-
-		this.buffer.markCollectorCompleted(this);
-	}
-
-	@Override
-	public void collect(Throwable error)  {
-		this.error = error;
-
-		this.buffer.markCollectorCompleted(this);
-	}
-
-	public List<OUT> getResult() throws IOException {
-		if (error != null) {
-			throw new IOException(error.getMessage());
-		}
-		return result;
-	}
-
-	public void markDone() {
-		isDone = true;
-	}
-
-	public boolean isDone() {
-		return isDone;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java
deleted file mode 100644
index 8883a2d..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/buffer/WatermarkEntry.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.async.buffer;
-
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-/**
- * {@link AsyncCollectorBuffer} entry for {@link Watermark}
- *
- */
-public class WatermarkEntry<OUT> extends AbstractBufferEntry<OUT> {
-	public WatermarkEntry(Watermark watermark) {
-		super(watermark);
-	}
-
-	@Override
-	public boolean isDone() {
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
index b2a58d2..a072aca 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/collector/AsyncCollector.java
@@ -18,16 +18,16 @@
 
 package org.apache.flink.streaming.api.functions.async.collector;
 
-import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
 
-import java.util.List;
+import java.util.Collection;
 
 /**
  * {@link AsyncCollector} collects data / error in user codes while processing async i/o.
  *
  * @param <OUT> Output type
  */
-@Internal
+@PublicEvolving
 public interface AsyncCollector<OUT> {
 	/**
 	 * Set result.
@@ -35,14 +35,15 @@ public interface AsyncCollector<OUT> {
 	 * Note that it should be called for exactly one time in the user code.
 	 * Calling this function for multiple times will cause data lose.
 	 * <p>
-	 * Put all results in a {@link List} and then issue {@link AsyncCollector#collect(List)}.
+	 * Put all results in a {@link Collection} and then issue
+	 * {@link AsyncCollector#collect(Collection)}.
 	 * <p>
 	 * If the result is NULL, it will cause task fail. If collecting empty result set is allowable and
 	 * should not cause task fail-over, then try to collect an empty list collection.
 	 *
 	 * @param result A list of results.
 	 */
-	void collect(List<OUT> result);
+	void collect(Collection<OUT> result);
 
 	/**
 	 * Set error

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java
index 56fa14d..dc80e81 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java
@@ -63,6 +63,10 @@ public class TimestampedCollector<T> implements Collector<T> {
 		reuse.setTimestamp(timestamp);
 	}
 
+	public void eraseTimestamp() {
+		reuse.eraseTimestamp();
+	}
+
 	@Override
 	public void close() {
 		output.close();

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
index 9166865..88fc833 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
@@ -19,110 +19,154 @@
 package org.apache.flink.streaming.api.operators.async;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.StateSnapshotContext;
 import org.apache.flink.streaming.api.datastream.AsyncDataStream;
-import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
-import org.apache.flink.streaming.api.functions.async.buffer.AsyncCollectorBuffer;
 import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.ChainingStrategy;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.async.queue.StreamElementQueue;
+import org.apache.flink.streaming.api.operators.async.queue.StreamElementQueueEntry;
+import org.apache.flink.streaming.api.operators.async.queue.StreamRecordQueueEntry;
+import org.apache.flink.streaming.api.operators.async.queue.WatermarkQueueEntry;
+import org.apache.flink.streaming.api.operators.async.queue.OrderedStreamElementQueue;
+import org.apache.flink.streaming.api.operators.async.queue.UnorderedStreamElementQueue;
 import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.Preconditions;
 
-import java.util.Iterator;
+import java.util.Collection;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 /**
- * The {@link AsyncWaitOperator} will accept input {@link StreamElement} from previous operators,
- * pass them into {@link AsyncFunction}, make a snapshot for the inputs in the {@link AsyncCollectorBuffer}
- * while checkpointing, and restore the {@link AsyncCollectorBuffer} from previous state.
+ * The {@link AsyncWaitOperator} allows to asynchronously process incoming stream records. For that
+ * the operator creates an {@link AsyncCollector} which is passed to an {@link AsyncFunction}.
+ * Within the async function, the user can complete the async collector arbitrarily. Once the async
+ * collector has been completed, the result is emitted by the operator's emitter to downstream
+ * operators.
  * <p>
- * Note that due to newly added working thread, named {@link AsyncCollectorBuffer.Emitter},
- * if {@link AsyncWaitOperator} is chained with other operators, {@link StreamTask} has to make sure that
- * the the order to open operators in the operator chain should be from the tail operator to the head operator,
- * and order to close operators in the operator chain should be from the head operator to the tail operator.
+ * The operator offers different output modes depending on the chosen
+ * {@link AsyncDataStream.OutputMode}. In order to give exactly once processing guarantees, the
+ * operator stores all currently in-flight {@link StreamElement} in it's operator state. Upon
+ * recovery the recorded set of stream elements is replayed.
+ * <p>
+ * In case of chaining of this operator, it has to be made sure that the operators in the chain are
+ * opened tail to head. The reason for this is that an opened {@link AsyncWaitOperator} starts
+ * already emitting recovered {@link StreamElement} to downstream operators.
  *
  * @param <IN> Input type for the operator.
  * @param <OUT> Output type for the operator.
  */
 @Internal
 public class AsyncWaitOperator<IN, OUT>
-	extends AbstractUdfStreamOperator<OUT, AsyncFunction<IN, OUT>>
-	implements OneInputStreamOperator<IN, OUT>
-{
+		extends AbstractUdfStreamOperator<OUT, AsyncFunction<IN, OUT>>
+		implements OneInputStreamOperator<IN, OUT>, OperatorActions {
 	private static final long serialVersionUID = 1L;
 
-	private final static String STATE_NAME = "_async_wait_operator_state_";
+	private static final String STATE_NAME = "_async_wait_operator_state_";
 
-	/**
-	 * {@link TypeSerializer} for inputs while making snapshots.
-	 */
+	/** Capacity of the stream element queue */
+	private final int capacity;
+
+	/** Output mode for this operator */
+	private final AsyncDataStream.OutputMode outputMode;
+
+	/** Timeout for the async collectors */
+	private final long timeout;
+
+	private transient Object checkpointingLock;
+
+	/** {@link TypeSerializer} for inputs while making snapshots. */
 	private transient StreamElementSerializer<IN> inStreamElementSerializer;
 
-	/**
-	 * input stream elements from the state
-	 */
+	/** Recovered input stream elements */
 	private transient ListState<StreamElement> recoveredStreamElements;
 
-	private transient TimestampedCollector<OUT> collector;
+	/** Queue to store the currently in-flight stream elements into */
+	private transient StreamElementQueue queue;
 
-	private transient AsyncCollectorBuffer<IN, OUT> buffer;
+	/** Pending stream element which could not yet added to the queue */
+	private transient StreamElementQueueEntry<?> pendingStreamElementQueueEntry;
 
-	/**
-	 * Checkpoint lock from {@link StreamTask#lock}
-	 */
-	private transient Object checkpointLock;
+	private transient ExecutorService executor;
+
+	/** Emitter for the completed stream element queue entries */
+	private transient Emitter<OUT> emitter;
 
-	private final int bufferSize;
-	private final AsyncDataStream.OutputMode mode;
+	/** Thread running the emitter */
+	private transient Thread emitterThread;
 
 
-	public AsyncWaitOperator(AsyncFunction<IN, OUT> asyncFunction, int bufferSize, AsyncDataStream.OutputMode mode) {
+	public AsyncWaitOperator(
+			AsyncFunction<IN, OUT> asyncFunction,
+			int capacity,
+			AsyncDataStream.OutputMode outputMode) {
 		super(asyncFunction);
 		chainingStrategy = ChainingStrategy.ALWAYS;
 
-		Preconditions.checkArgument(bufferSize > 0, "The number of concurrent async operation should be greater than 0.");
-		this.bufferSize = bufferSize;
+		Preconditions.checkArgument(capacity > 0, "The number of concurrent async operation should be greater than 0.");
+		this.capacity = capacity;
 
-		this.mode = mode;
+		this.outputMode = Preconditions.checkNotNull(outputMode, "outputMode");
+
+		this.timeout = -1L;
 	}
 
 	@Override
 	public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output) {
 		super.setup(containingTask, config, output);
 
-		this.inStreamElementSerializer =
-				new StreamElementSerializer(this.getOperatorConfig().<IN>getTypeSerializerIn1(getUserCodeClassloader()));
-
-		this.collector = new TimestampedCollector<>(output);
-
-		this.checkpointLock = containingTask.getCheckpointLock();
-
-		this.buffer = new AsyncCollectorBuffer<>(bufferSize, mode, output, collector, this.checkpointLock, this);
+		this.checkpointingLock = getContainingTask().getCheckpointLock();
+
+		this.inStreamElementSerializer = new StreamElementSerializer<>(
+			getOperatorConfig().<IN>getTypeSerializerIn1(getUserCodeClassloader()));
+
+		// create the operators executor for the complete operations of the queue entries
+		this.executor = Executors.newSingleThreadExecutor();
+
+		switch (outputMode) {
+			case ORDERED:
+				queue = new OrderedStreamElementQueue(
+					capacity,
+					executor,
+					this);
+				break;
+			case UNORDERED:
+				queue = new UnorderedStreamElementQueue(
+					capacity,
+					executor,
+					this);
+				break;
+			default:
+				throw new IllegalStateException("Unknown async mode: " + outputMode + '.');
+		}
 	}
 
 	@Override
 	public void open() throws Exception {
 		super.open();
 
-		// process stream elements from state, since the Emit thread will start soon as all elements from
-		// previous state are in the AsyncCollectorBuffer, we have to make sure that the order to open all
-		// operators in the operator chain should be from the tail operator to the head operator.
-		if (this.recoveredStreamElements != null) {
-			for (StreamElement element : this.recoveredStreamElements.get()) {
+		// process stream elements from state, since the Emit thread will start as soon as all
+		// elements from previous state are in the StreamElementQueue, we have to make sure that the
+		// order to open all operators in the operator chain proceeds from the tail operator to the
+		// head operator.
+		if (recoveredStreamElements != null) {
+			for (StreamElement element : recoveredStreamElements.get()) {
 				if (element.isRecord()) {
 					processElement(element.<IN>asRecord());
 				}
@@ -133,30 +177,52 @@ public class AsyncWaitOperator<IN, OUT>
 					processLatencyMarker(element.asLatencyMarker());
 				}
 				else {
-					throw new Exception("Unknown record type: "+element.getClass());
+					throw new IllegalStateException("Unknown record type " + element.getClass() +
+						" encountered while opening the operator.");
 				}
 			}
-			this.recoveredStreamElements = null;
+			recoveredStreamElements = null;
 		}
 
-		buffer.startEmitterThread();
+		// create the emitter
+		this.emitter = new Emitter<>(checkpointingLock, output, queue, this);
+
+		// start the emitter thread
+		this.emitterThread = new Thread(emitter);
+		emitterThread.setDaemon(true);
+		emitterThread.start();
+
 	}
 
 	@Override
 	public void processElement(StreamRecord<IN> element) throws Exception {
-		AsyncCollector<OUT> collector = buffer.addStreamRecord(element);
+		final StreamRecordQueueEntry<OUT> streamRecordBufferEntry = new StreamRecordQueueEntry<>(element);
+
+		if (timeout > 0L) {
+			// register a timeout for this AsyncStreamRecordBufferEntry
+			long timeoutTimestamp = timeout + System.currentTimeMillis();
+
+			getProcessingTimeService().registerTimer(
+				timeoutTimestamp,
+				new ProcessingTimeCallback() {
+					@Override
+					public void onProcessingTime(long timestamp) throws Exception {
+						streamRecordBufferEntry.collect(
+							new TimeoutException("Async function call has timed out."));
+					}
+				});
+		}
 
-		userFunction.asyncInvoke(element.getValue(), collector);
+		addAsyncBufferEntry(streamRecordBufferEntry);
+
+		userFunction.asyncInvoke(element.getValue(), streamRecordBufferEntry);
 	}
 
 	@Override
 	public void processWatermark(Watermark mark) throws Exception {
-		buffer.addWatermark(mark);
-	}
+		WatermarkQueueEntry watermarkBufferEntry = new WatermarkQueueEntry(mark);
 
-	@Override
-	public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception {
-		buffer.addLatencyMarker(latencyMarker);
+		addAsyncBufferEntry(watermarkBufferEntry);
 	}
 
 	@Override
@@ -167,45 +233,155 @@ public class AsyncWaitOperator<IN, OUT>
 				getOperatorStateBackend().getOperatorState(new ListStateDescriptor<>(STATE_NAME, inStreamElementSerializer));
 		partitionableState.clear();
 
-		Iterator<StreamElement> iterator = buffer.getStreamElementsInBuffer();
-		while (iterator.hasNext()) {
-			partitionableState.add(iterator.next());
+		Collection<StreamElementQueueEntry<?>> values = queue.values();
+
+		for (StreamElementQueueEntry<?> value : values) {
+			partitionableState.add(value.getStreamElement());
+		}
+
+		// add the pending stream element queue entry if the stream element queue is currently full
+		if (pendingStreamElementQueueEntry != null) {
+			partitionableState.add(pendingStreamElementQueueEntry.getStreamElement());
 		}
 	}
 
 	@Override
 	public void initializeState(StateInitializationContext context) throws Exception {
-		recoveredStreamElements =
-				context.getOperatorStateStore().getOperatorState(new ListStateDescriptor<>(STATE_NAME, inStreamElementSerializer));
+		recoveredStreamElements = context
+			.getOperatorStateStore()
+			.getOperatorState(new ListStateDescriptor<>(STATE_NAME, inStreamElementSerializer));
 
 	}
 
 	@Override
 	public void close() throws Exception {
 		try {
-			buffer.waitEmpty();
+			assert(Thread.holdsLock(checkpointingLock));
+
+			while (!queue.isEmpty()) {
+				// wait for the emitter thread to output the remaining elements
+				// for that he needs the checkpointing lock and thus we have to free it
+				checkpointingLock.wait();
+			}
 		}
 		finally {
-			// make sure Emitter thread exits and close user function
-			buffer.stopEmitterThread();
+			Exception exception = null;
+
+			try {
+				super.close();
+			} catch (InterruptedException interrupted) {
+				exception = interrupted;
+
+				Thread.currentThread().interrupt();
+			} catch (Exception e) {
+				exception = e;
+			}
+
+			try {
+				// terminate the emitter, the emitter thread and the executor
+				stopResources(true);
+			} catch (InterruptedException interrupted) {
+				exception = ExceptionUtils.firstOrSuppressed(interrupted, exception);
+
+				Thread.currentThread().interrupt();
+			} catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
 
-			super.close();
+			if (exception != null) {
+				LOG.warn("Errors occurred while closing the AsyncWaitOperator.", exception);
+			}
 		}
 	}
 
 	@Override
 	public void dispose() throws Exception {
-		super.dispose();
+		Exception exception = null;
+
+		try {
+			super.dispose();
+		} catch (InterruptedException interrupted) {
+			exception = interrupted;
+
+			Thread.currentThread().interrupt();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		try {
+			stopResources(false);
+		} catch (InterruptedException interrupted) {
+			exception = ExceptionUtils.firstOrSuppressed(interrupted, exception);
+
+			Thread.currentThread().interrupt();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw exception;
+		}
+	}
 
-		buffer.stopEmitterThread();
+	/**
+	 * Close the operator's resources. They include the emitter thread and the executor to run
+	 * the queue's complete operation.
+	 *
+	 * @param waitForShutdown is true if the method should wait for the resources to be freed;
+	 *                           otherwise false.
+	 * @throws InterruptedException if current thread has been interrupted
+	 */
+	private void stopResources(boolean waitForShutdown) throws InterruptedException {
+		emitter.stop();
+		emitterThread.interrupt();
+
+		executor.shutdown();
+
+		if (waitForShutdown) {
+			try {
+				if (!executor.awaitTermination(365L, TimeUnit.DAYS)) {
+					executor.shutdownNow();
+				}
+			} catch (InterruptedException e) {
+				executor.shutdownNow();
+
+				Thread.currentThread().interrupt();
+			}
+
+			emitterThread.join();
+		} else {
+			executor.shutdownNow();
+		}
 	}
 
-	public void sendLatencyMarker(LatencyMarker marker) throws Exception {
-		super.processLatencyMarker(marker);
+	/**
+	 * Add the given stream element queue entry to the operator's stream element queue. This
+	 * operation blocks until the element has been added.
+	 * <p>
+	 * For that it tries to put the element into the queue and if not successful then it waits on
+	 * the checkpointing lock. The checkpointing lock is also used by the {@link Emitter} to output
+	 * elements. The emitter is also responsible for notifying this method if the queue has capacity
+	 * left again, by calling notifyAll on the checkpointing lock.
+	 *
+	 * @param streamElementQueueEntry to add to the operator's queue
+	 * @param <T> Type of the stream element queue entry's result
+	 * @throws InterruptedException if the current thread has been interrupted
+	 */
+	private <T> void addAsyncBufferEntry(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException {
+		assert(Thread.holdsLock(checkpointingLock));
+
+		pendingStreamElementQueueEntry = streamElementQueueEntry;
+
+		while (!queue.tryPut(streamElementQueueEntry)) {
+			// we wait for the emitter to notify us if the queue has space left again
+			checkpointingLock.wait();
+		}
+
+		pendingStreamElementQueueEntry = null;
 	}
 
-	@VisibleForTesting
-	public AsyncCollectorBuffer<IN, OUT> getBuffer() {
-		return buffer;
+	@Override
+	public void failOperator(Throwable throwable) {
+		getContainingTask().getEnvironment().failExternally(throwable);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
new file mode 100644
index 0000000..4b22aaa
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
@@ -0,0 +1,152 @@
+/*
+ * 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.async;
+
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.async.queue.AsyncCollectionResult;
+import org.apache.flink.streaming.api.operators.async.queue.StreamElementQueue;
+import org.apache.flink.streaming.api.operators.async.queue.AsyncResult;
+import org.apache.flink.streaming.api.operators.async.queue.AsyncWatermarkResult;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+
+/**
+ * Runnable responsible for consuming elements from the given queue and outputting them to the
+ * given output/timestampedCollector.
+ *
+ * @param <OUT> Type of the output elements
+ */
+public class Emitter<OUT> implements Runnable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(Emitter.class);
+
+	/** Lock to hold before outputting */
+	private final Object checkpointLock;
+
+	/** Output for the watermark elements */
+	private final Output<StreamRecord<OUT>> output;
+
+	/** Queue to consume the async results from */
+	private final StreamElementQueue streamElementQueue;
+
+	private final OperatorActions operatorActions;
+
+	/** Output for stream records */
+	private final TimestampedCollector<OUT> timestampedCollector;
+
+	private volatile boolean running;
+
+	public Emitter(
+			final Object checkpointLock,
+			final Output<StreamRecord<OUT>> output,
+			final StreamElementQueue streamElementQueue,
+			final OperatorActions operatorActions) {
+
+		this.checkpointLock = Preconditions.checkNotNull(checkpointLock, "checkpointLock");
+		this.output = Preconditions.checkNotNull(output, "output");
+		this.streamElementQueue = Preconditions.checkNotNull(streamElementQueue, "asyncCollectorBuffer");
+		this.operatorActions = Preconditions.checkNotNull(operatorActions, "operatorActions");
+
+		this.timestampedCollector = new TimestampedCollector<>(this.output);
+		this.running = true;
+	}
+
+	@Override
+	public void run() {
+		try {
+			while (running) {
+				LOG.debug("Wait for next completed async stream element result.");
+				AsyncResult streamElementEntry = streamElementQueue.peekBlockingly();
+
+				output(streamElementEntry);
+			}
+		} catch (InterruptedException e) {
+			if (running) {
+				operatorActions.failOperator(e);
+			} else {
+				// Thread got interrupted which means that it should shut down
+				LOG.debug("Emitter thread got interrupted. This indicates that the emitter should " +
+					"shut down.");
+			}
+		} catch (Throwable t) {
+			operatorActions.failOperator(new Exception("AsyncWaitOperator's emitter caught an " +
+				"unexpected throwable.", t));
+		}
+	}
+
+	private void output(AsyncResult asyncResult) throws InterruptedException {
+		if (asyncResult.isWatermark()) {
+			synchronized (checkpointLock) {
+				// remove the peeked element from the async collector buffer so that it is no longer
+				// checkpointed
+				streamElementQueue.poll();
+
+				// notify the main thread that there is again space left in the async collector
+				// buffer
+				checkpointLock.notifyAll();
+
+				AsyncWatermarkResult asyncWatermarkResult = asyncResult.asWatermark();
+
+				LOG.debug("Output async watermark.");
+				output.emitWatermark(asyncWatermarkResult.getWatermark());
+			}
+		} else {
+			AsyncCollectionResult<OUT> streamRecordResult = asyncResult.asResultCollection();
+
+			if (streamRecordResult.hasTimestamp()) {
+				timestampedCollector.setAbsoluteTimestamp(streamRecordResult.getTimestamp());
+			} else {
+				timestampedCollector.eraseTimestamp();
+			}
+
+			synchronized (checkpointLock) {
+				// remove the peeked element from the async collector buffer so that it is no longer
+				// checkpointed
+				streamElementQueue.poll();
+
+				// notify the main thread that there is again space left in the async collector
+				// buffer
+				checkpointLock.notifyAll();
+
+				LOG.debug("Output async stream element collection result.");
+
+				try {
+					Collection<OUT> resultCollection = streamRecordResult.get();
+
+					for (OUT result : resultCollection) {
+						timestampedCollector.collect(result);
+					}
+				} catch (Exception e) {
+					operatorActions.failOperator(
+						new Exception("An async function call terminated with an exception. " +
+							"Failing the AsyncWaitOperator.", e));
+				}
+			}
+		}
+	}
+
+	public void stop() {
+		running = false;
+	}
+}


[3/7] flink git commit: [FLINK-4391] Polish asynchronous I/O operations

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
index 560ee5a..10ee14f 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.api.operators.async;
 
 import org.apache.flink.api.common.ExecutionConfig;
+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.api.common.typeutils.base.IntSerializer;
@@ -36,8 +37,10 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.async.RichAsyncFunction;
 import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.async.queue.StreamElementQueue;
+import org.apache.flink.streaming.api.operators.async.queue.StreamElementQueueEntry;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
@@ -45,17 +48,15 @@ import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
 import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.util.ArrayDeque;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Queue;
-import java.util.Random;
-import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
@@ -74,19 +75,16 @@ import static org.junit.Assert.assertEquals;
  *     <li>Snapshot state and restore state</li>
  * </ul>
  */
-public class AsyncWaitOperatorTest {
-
-	// hold sink result
-	private static Queue<Object> sinkResult;
+public class AsyncWaitOperatorTest extends TestLogger {
 
 	private static class MyAsyncFunction extends RichAsyncFunction<Integer, Integer> {
-		final int SLEEP_FACTOR = 100;
-		final int THREAD_POOL_SIZE = 10;
+		private static final long serialVersionUID = 8522411971886428444L;
 
-		transient static ExecutorService executorService;
-		static int counter = 0;
+		private static final long TIMEOUT = 5000L;
+		private static final int THREAD_POOL_SIZE = 10;
 
-		static Random random = new Random();
+		static ExecutorService executorService;
+		static int counter = 0;
 
 		@Override
 		public void open(Configuration parameters) throws Exception {
@@ -105,33 +103,35 @@ public class AsyncWaitOperatorTest {
 		public void close() throws Exception {
 			super.close();
 
+			freeExecutor();
+		}
+
+		private void freeExecutor() {
 			synchronized (MyAsyncFunction.class) {
 				--counter;
 
 				if (counter == 0) {
 					executorService.shutdown();
-					executorService.awaitTermination(SLEEP_FACTOR * THREAD_POOL_SIZE, TimeUnit.MILLISECONDS);
+
+					try {
+						if (!executorService.awaitTermination(TIMEOUT, TimeUnit.MILLISECONDS)) {
+							executorService.shutdownNow();
+						}
+					} catch (InterruptedException interrupted) {
+						executorService.shutdownNow();
+
+						Thread.currentThread().interrupt();
+					}
 				}
 			}
 		}
 
 		@Override
 		public void asyncInvoke(final Integer input, final AsyncCollector<Integer> collector) throws Exception {
-			this.executorService.submit(new Runnable() {
+			executorService.submit(new Runnable() {
 				@Override
 				public void run() {
-					// wait for while to simulate async operation here
-					int sleep = (int) (random.nextFloat() * SLEEP_FACTOR);
-
-					try {
-						Thread.sleep(sleep);
-						List<Integer> ret = new ArrayList<>();
-						ret.add(input*2);
-						collector.collect(ret);
-					}
-					catch (InterruptedException e) {
-						// do nothing
-					}
+					collector.collect(Collections.singletonList(input * 2));
 				}
 			});
 		}
@@ -141,11 +141,13 @@ public class AsyncWaitOperatorTest {
 	 * A special {@link org.apache.flink.streaming.api.functions.async.AsyncFunction} without issuing
 	 * {@link AsyncCollector#collect} until the latch counts to zero.
 	 * This function is used in the testStateSnapshotAndRestore, ensuring
-	 * that {@link org.apache.flink.streaming.api.functions.async.buffer.StreamElementEntry} can stay
-	 * in the {@link org.apache.flink.streaming.api.functions.async.buffer.AsyncCollectorBuffer} to be
+	 * that {@link StreamElementQueueEntry} can stay
+	 * in the {@link StreamElementQueue} to be
 	 * snapshotted while checkpointing.
 	 */
 	private static class LazyAsyncFunction extends MyAsyncFunction {
+		private static final long serialVersionUID = 3537791752703154670L;
+
 		private static CountDownLatch latch;
 
 		public LazyAsyncFunction() {
@@ -200,17 +202,23 @@ public class AsyncWaitOperatorTest {
 		}
 	}
 
+	/**
+	 * Test the AsyncWaitOperator with ordered mode and event time.
+	 */
 	@Test
-	public void testWaterMarkOrdered() throws Exception {
-		testWithWatermark(AsyncDataStream.OutputMode.ORDERED);
+	public void testEventTimeOrdered() throws Exception {
+		testEventTime(AsyncDataStream.OutputMode.ORDERED);
 	}
 
+	/**
+	 * Test the AsyncWaitOperator with unordered mode and event time.
+	 */
 	@Test
 	public void testWaterMarkUnordered() throws Exception {
-		testWithWatermark(AsyncDataStream.OutputMode.UNORDERED);
+		testEventTime(AsyncDataStream.OutputMode.UNORDERED);
 	}
 
-	private void testWithWatermark(AsyncDataStream.OutputMode mode) throws Exception {
+	private void testEventTime(AsyncDataStream.OutputMode mode) throws Exception {
 		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(new MyAsyncFunction(), 2, mode);
 
 		final OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
@@ -255,40 +263,42 @@ public class AsyncWaitOperatorTest {
 		}
 	}
 
+	/**
+	 * Test the AsyncWaitOperator with ordered mode and processing time.
+	 */
 	@Test
-	public void testOrdered() throws Exception {
-		testRun(AsyncDataStream.OutputMode.ORDERED);
+	public void testProcessingTimeOrdered() throws Exception {
+		testProcessingTime(AsyncDataStream.OutputMode.ORDERED);
 	}
 
+	/**
+	 * Test the AsyncWaitOperator with unordered mode and processing time.
+	 */
 	@Test
-	public void testUnordered() throws Exception {
-		testRun(AsyncDataStream.OutputMode.UNORDERED);
+	public void testProcessingUnordered() throws Exception {
+		testProcessingTime(AsyncDataStream.OutputMode.UNORDERED);
 	}
 
-	private void testRun(AsyncDataStream.OutputMode mode) throws Exception {
-		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
-		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
-				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
-
+	private void testProcessingTime(AsyncDataStream.OutputMode mode) throws Exception {
 		final AsyncWaitOperator<Integer, Integer> operator = new AsyncWaitOperator<>(new MyAsyncFunction(), 6, mode);
 
-		final StreamConfig streamConfig = testHarness.getStreamConfig();
-		streamConfig.setStreamOperator(operator);
-
-		testHarness.invoke();
-		testHarness.waitForTaskRunning();
+		final OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator, IntSerializer.INSTANCE);
 
 		final long initialTime = 0L;
-		final ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
+		final Queue<Object> expectedOutput = new ArrayDeque<>();
 
-		testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
-		testHarness.processElement(new StreamRecord<>(2, 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));
+		testHarness.open();
+
+		synchronized (testHarness.getCheckpointLock()) {
+			testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
+			testHarness.processElement(new StreamRecord<>(2, 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 + 1));
 		expectedOutput.add(new StreamRecord<>(4, initialTime + 2));
@@ -299,11 +309,9 @@ public class AsyncWaitOperatorTest {
 		expectedOutput.add(new StreamRecord<>(14, initialTime + 7));
 		expectedOutput.add(new StreamRecord<>(16, initialTime + 8));
 
-		testHarness.waitForInputProcessing();
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
+		synchronized (testHarness.getCheckpointLock()) {
+			testHarness.close();
+		}
 
 		if (mode == AsyncDataStream.OutputMode.ORDERED) {
 			TestHarnessUtil.assertOutputEquals("ORDERED Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -317,6 +325,54 @@ public class AsyncWaitOperatorTest {
 		}
 	}
 
+	/**
+	 *	Tests that the AsyncWaitOperator works together with chaining
+	 */
+	@Test
+	public void testOperatorChainWithProcessingTime() throws Exception {
+
+		JobVertex chainedVertex = createChainedVertex(false);
+
+		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
+		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
+				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
+
+		testHarness.taskConfig = chainedVertex.getConfiguration();
+
+		final StreamConfig streamConfig = testHarness.getStreamConfig();
+		final StreamConfig operatorChainStreamConfig = new StreamConfig(chainedVertex.getConfiguration());
+		final AsyncWaitOperator<Integer, Integer> headOperator =
+				operatorChainStreamConfig.getStreamOperator(AsyncWaitOperatorTest.class.getClassLoader());
+		streamConfig.setStreamOperator(headOperator);
+
+		testHarness.invoke();
+		testHarness.waitForTaskRunning();
+
+		long initialTimestamp = 0L;
+
+		testHarness.processElement(new StreamRecord<>(5, initialTimestamp));
+		testHarness.processElement(new StreamRecord<>(6, initialTimestamp + 1L));
+		testHarness.processElement(new StreamRecord<>(7, initialTimestamp + 2L));
+		testHarness.processElement(new StreamRecord<>(8, initialTimestamp + 3L));
+		testHarness.processElement(new StreamRecord<>(9, initialTimestamp + 4L));
+
+		testHarness.endInput();
+		testHarness.waitForTaskCompletion();
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+		expectedOutput.add(new StreamRecord<>(22, initialTimestamp));
+		expectedOutput.add(new StreamRecord<>(26, initialTimestamp + 1L));
+		expectedOutput.add(new StreamRecord<>(30, initialTimestamp + 2L));
+		expectedOutput.add(new StreamRecord<>(34, initialTimestamp + 3L));
+		expectedOutput.add(new StreamRecord<>(38, initialTimestamp + 4L));
+
+		TestHarnessUtil.assertOutputEqualsSorted(
+				"Test for chained operator with AsyncWaitOperator failed",
+				expectedOutput,
+				testHarness.getOutput(),
+				new StreamRecordComparator());
+	}
+
 	private JobVertex createChainedVertex(boolean withLazyFunction) {
 		StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -353,144 +409,23 @@ public class AsyncWaitOperatorTest {
 
 		input = AsyncDataStream.unorderedWait(input, new MyAsyncFunction(), 3);
 
-		input.addSink(new SinkFunction<Integer>() {
-			private static final long serialVersionUID = 1L;
+		input.map(new MapFunction<Integer, Integer>() {
+			private static final long serialVersionUID = 5162085254238405527L;
 
 			@Override
-			public void invoke(Integer value) throws Exception {
-				sinkResult.add(value);
+			public Integer map(Integer value) throws Exception {
+				return value;
 			}
-		});
+		}).startNewChain().addSink(new DiscardingSink<Integer>());
 
 		// be build our own OperatorChain
 		final JobGraph jobGraph = chainEnv.getStreamGraph().getJobGraph();
 
-		Assert.assertTrue(jobGraph.getVerticesSortedTopologicallyFromSources().size() == 2);
+		Assert.assertTrue(jobGraph.getVerticesSortedTopologicallyFromSources().size() == 3);
 
 		return jobGraph.getVerticesSortedTopologicallyFromSources().get(1);
 	}
 
-	/**
-	 * Get the {@link SubtaskState} for the operator chain. The state will keep several inputs.
-	 *
-	 * @return A {@link SubtaskState}
-	 * @throws Exception
-     */
-	private SubtaskState createTaskState() throws Exception {
-		sinkResult = new ConcurrentLinkedDeque<>();
-
-		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
-		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
-				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
-
-		JobVertex chainedVertex = createChainedVertex(true);
-
-		testHarness.taskConfig = chainedVertex.getConfiguration();
-
-		final AcknowledgeStreamMockEnvironment env = new AcknowledgeStreamMockEnvironment(
-				testHarness.jobConfig,
-				testHarness.taskConfig,
-				testHarness.getExecutionConfig(),
-				testHarness.memorySize,
-				new MockInputSplitProvider(),
-				testHarness.bufferSize);
-
-		final StreamConfig streamConfig = testHarness.getStreamConfig();
-		final StreamConfig operatorChainStreamConfig = new StreamConfig(chainedVertex.getConfiguration());
-		final AsyncWaitOperator<Integer, Integer> headOperator =
-				operatorChainStreamConfig.getStreamOperator(Thread.currentThread().getContextClassLoader());
-		streamConfig.setStreamOperator(headOperator);
-
-		testHarness.invoke(env);
-		testHarness.waitForTaskRunning();
-
-		testHarness.processElement(new StreamRecord<>(1));
-		testHarness.processElement(new StreamRecord<>(2));
-		testHarness.processElement(new StreamRecord<>(3));
-		testHarness.processElement(new StreamRecord<>(4));
-
-		testHarness.waitForInputProcessing();
-
-		final CheckpointMetaData checkpointMetaData = new CheckpointMetaData(1L, 1L);
-
-		task.triggerCheckpoint(checkpointMetaData);
-
-		env.getCheckpointLatch().await();
-
-		assertEquals(1L, env.getCheckpointId());
-
-		LazyAsyncFunction.countDown();
-
-		testHarness.endInput();
-		testHarness.waitForTaskCompletion();
-
-		return env.getCheckpointStateHandles();
-	}
-
-	@Test
-	public void testOperatorChain() throws Exception {
-
-		JobVertex chainedVertex = createChainedVertex(false);
-
-		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
-		final OneInputStreamTaskTestHarness<Integer, Integer> testHarness =
-				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
-
-		task.setInitialState(new TaskStateHandles(createTaskState()));
-
-		sinkResult = new ConcurrentLinkedDeque<>();
-
-		testHarness.taskConfig = chainedVertex.getConfiguration();
-
-		final AcknowledgeStreamMockEnvironment env = new AcknowledgeStreamMockEnvironment(
-				testHarness.jobConfig,
-				testHarness.taskConfig,
-				testHarness.getExecutionConfig(),
-				testHarness.memorySize,
-				new MockInputSplitProvider(),
-				testHarness.bufferSize);
-
-		final StreamConfig streamConfig = testHarness.getStreamConfig();
-		final StreamConfig operatorChainStreamConfig = new StreamConfig(chainedVertex.getConfiguration());
-		final AsyncWaitOperator<Integer, Integer> headOperator =
-				operatorChainStreamConfig.getStreamOperator(Thread.currentThread().getContextClassLoader());
-		streamConfig.setStreamOperator(headOperator);
-
-		testHarness.invoke(env);
-		testHarness.waitForTaskRunning();
-
-		testHarness.processElement(new StreamRecord<>(5));
-		testHarness.processElement(new StreamRecord<>(6));
-		testHarness.processElement(new StreamRecord<>(7));
-		testHarness.processElement(new StreamRecord<>(8));
-		testHarness.processElement(new StreamRecord<>(9));
-
-		testHarness.endInput();
-		testHarness.waitForTaskCompletion();
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-		expectedOutput.add(6);
-		expectedOutput.add(10);
-		expectedOutput.add(14);
-		expectedOutput.add(18);
-		expectedOutput.add(22);
-		expectedOutput.add(26);
-		expectedOutput.add(30);
-		expectedOutput.add(34);
-		expectedOutput.add(38);
-
-		TestHarnessUtil.assertOutputEqualsSorted(
-				"Test for chained operator with AsyncWaitOperator failed",
-				expectedOutput,
-				sinkResult,
-				new Comparator<Object>() {
-					@Override
-					public int compare(Object o1, Object o2) {
-						return (Integer)o1 - (Integer)o2;
-					}
-				});
-	}
-
 	@Test
 	public void testStateSnapshotAndRestore() throws Exception {
 		final OneInputStreamTask<Integer, Integer> task = new OneInputStreamTask<>();
@@ -498,7 +433,7 @@ public class AsyncWaitOperatorTest {
 				new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
 
 		AsyncWaitOperator<Integer, Integer> operator =
-				new AsyncWaitOperator<>(new LazyAsyncFunction(), 6, AsyncDataStream.OutputMode.ORDERED);
+				new AsyncWaitOperator<>(new LazyAsyncFunction(), 3, AsyncDataStream.OutputMode.ORDERED);
 
 		final StreamConfig streamConfig = testHarness.getStreamConfig();
 		streamConfig.setStreamOperator(operator);

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/EmitterTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/EmitterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/EmitterTest.java
new file mode 100644
index 0000000..c3a47aa
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/EmitterTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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.async;
+
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.async.queue.OrderedStreamElementQueue;
+import org.apache.flink.streaming.api.operators.async.queue.StreamElementQueue;
+import org.apache.flink.streaming.api.operators.async.queue.StreamRecordQueueEntry;
+import org.apache.flink.streaming.api.operators.async.queue.WatermarkQueueEntry;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.CollectorOutput;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class EmitterTest extends TestLogger {
+
+	private static final long timeout = 10000L;
+	private static ExecutorService executor;
+
+	@BeforeClass
+	public static void setup() {
+		executor = Executors.newFixedThreadPool(3);
+	}
+
+	@AfterClass
+	public static void shutdown() {
+		executor.shutdown();
+
+		try {
+			if (!executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)) {
+				executor.shutdownNow();
+			}
+		} catch (InterruptedException interrupted) {
+			executor.shutdownNow();
+
+			Thread.currentThread().interrupt();
+		}
+	}
+
+	/**
+	 * Tests that the emitter outputs completed stream element queue entries.
+	 */
+	@Test
+	public void testEmitterWithOrderedQueue() throws Exception {
+		Object lock = new Object();
+		List<StreamElement> list = new ArrayList<>();
+		Output<StreamRecord<Integer>> output = new CollectorOutput<>(list);
+
+		List<StreamElement> expected = Arrays.asList(
+			new StreamRecord<>(1, 0L),
+			new StreamRecord<>(2, 0L),
+			new StreamRecord<>(3, 1L),
+			new StreamRecord<>(4, 1L),
+			new Watermark(3L),
+			new StreamRecord<>(5, 4L),
+			new StreamRecord<>(6, 4L));
+
+		OperatorActions operatorActions = mock(OperatorActions.class);
+
+		final int capacity = 5;
+
+		StreamElementQueue queue = new OrderedStreamElementQueue(capacity, executor, operatorActions);
+
+		final Emitter<Integer> emitter = new Emitter<>(lock, output, queue, operatorActions);
+
+		final Thread emitterThread = new Thread(emitter);
+		emitterThread.start();
+
+		try {
+			StreamRecordQueueEntry<Integer> record1 = new StreamRecordQueueEntry<>(new StreamRecord<>(1, 0L));
+			StreamRecordQueueEntry<Integer> record2 = new StreamRecordQueueEntry<>(new StreamRecord<>(2, 1L));
+			WatermarkQueueEntry watermark1 = new WatermarkQueueEntry(new Watermark(3L));
+			StreamRecordQueueEntry<Integer> record3 = new StreamRecordQueueEntry<>(new StreamRecord<>(3, 4L));
+
+			queue.put(record1);
+			queue.put(record2);
+			queue.put(watermark1);
+			queue.put(record3);
+
+			record2.collect(Arrays.asList(3, 4));
+			record1.collect(Arrays.asList(1, 2));
+			record3.collect(Arrays.asList(5, 6));
+
+			synchronized (lock) {
+				while (!queue.isEmpty()) {
+					lock.wait();
+				}
+			}
+
+			Assert.assertEquals(expected, list);
+		} finally {
+			emitter.stop();
+			emitterThread.interrupt();
+		}
+	}
+
+	/**
+	 * Tests that the emitter handles exceptions occurring in the {@link AsyncCollector} correctly.
+	 */
+	@Test
+	public void testEmitterWithExceptions() throws Exception {
+		Object lock = new Object();
+		List<StreamElement> list = new ArrayList<>();
+		Output<StreamRecord<Integer>> output = new CollectorOutput<>(list);
+
+		List<StreamElement> expected = Arrays.asList(
+			new StreamRecord<>(1, 0L),
+			new Watermark(3L));
+
+		OperatorActions operatorActions = mock(OperatorActions.class);
+
+		final int capacity = 3;
+
+		StreamElementQueue queue = new OrderedStreamElementQueue(capacity, executor, operatorActions);
+
+		final Emitter<Integer> emitter = new Emitter<>(lock, output, queue, operatorActions);
+
+		final Thread emitterThread = new Thread(emitter);
+		emitterThread.start();
+
+		final Exception testException = new Exception("Test exception");
+
+		try {
+			StreamRecordQueueEntry<Integer> record1 = new StreamRecordQueueEntry<>(new StreamRecord<>(1, 0L));
+			StreamRecordQueueEntry<Integer> record2 = new StreamRecordQueueEntry<>(new StreamRecord<>(2, 1L));
+			WatermarkQueueEntry watermark1 = new WatermarkQueueEntry(new Watermark(3L));
+
+			queue.put(record1);
+			queue.put(record2);
+			queue.put(watermark1);
+
+			record2.collect(testException);
+			record1.collect(Arrays.asList(1));
+
+			synchronized (lock) {
+				while (!queue.isEmpty()) {
+					lock.wait();
+				}
+			}
+
+			Assert.assertEquals(expected, list);
+
+			ArgumentCaptor<Throwable> argumentCaptor = ArgumentCaptor.forClass(Throwable.class);
+
+			verify(operatorActions).failOperator(argumentCaptor.capture());
+
+			Throwable failureCause = argumentCaptor.getValue();
+
+			Assert.assertNotNull(failureCause.getCause());
+			Assert.assertTrue(failureCause.getCause() instanceof ExecutionException);
+
+			Assert.assertNotNull(failureCause.getCause().getCause());
+			Assert.assertEquals(testException, failureCause.getCause().getCause());
+		} finally {
+			emitter.stop();
+			emitterThread.interrupt();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java
new file mode 100644
index 0000000..0380512
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueueTest.java
@@ -0,0 +1,124 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.streaming.api.operators.async.OperatorActions;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+
+/**
+ * {@link OrderedStreamElementQueue} specific tests
+ */
+public class OrderedStreamElementQueueTest extends TestLogger {
+
+	private static final long timeout = 10000L;
+	private static ExecutorService executor;
+
+	@BeforeClass
+	public static void setup() {
+		executor = Executors.newFixedThreadPool(3);
+	}
+
+	@AfterClass
+	public static void shutdown() {
+		executor.shutdown();
+
+		try {
+			if (!executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)) {
+				executor.shutdownNow();
+			}
+		} catch (InterruptedException interrupted) {
+			executor.shutdownNow();
+
+			Thread.currentThread().interrupt();
+		}
+	}
+
+	/**
+	 * Tests that only the head element is pulled from the ordered queue if it has been
+	 * completed.
+	 */
+	@Test
+	public void testCompletionOrder() throws Exception {
+		OperatorActions operatorActions = mock(OperatorActions.class);
+		final OrderedStreamElementQueue queue = new OrderedStreamElementQueue(4, executor, operatorActions);
+
+		StreamRecordQueueEntry<Integer> entry1 = new StreamRecordQueueEntry<>(new StreamRecord<>(1, 0L));
+		StreamRecordQueueEntry<Integer> entry2 = new StreamRecordQueueEntry<>(new StreamRecord<>(2, 1L));
+		WatermarkQueueEntry entry3 = new WatermarkQueueEntry(new Watermark(2L));
+		StreamRecordQueueEntry<Integer> entry4 = new StreamRecordQueueEntry<>(new StreamRecord<>(3, 3L));
+
+		List<StreamElementQueueEntry<?>> expected = Arrays.asList(entry1, entry2, entry3, entry4);
+
+		for (StreamElementQueueEntry<?> entry : expected) {
+			queue.put(entry);
+		}
+
+		Future<List<AsyncResult>> pollOperation = FlinkFuture.supplyAsync(new Callable<List<AsyncResult>>() {
+			@Override
+			public List<AsyncResult> call() throws Exception {
+				List<AsyncResult> result = new ArrayList<>(4);
+				while (!queue.isEmpty()) {
+					result.add(queue.poll());
+				}
+
+				return result;
+			}
+		}, executor);
+
+		Thread.sleep(10L);
+
+		Assert.assertFalse(pollOperation.isDone());
+
+		entry2.collect(Collections.<Integer>emptyList());
+
+		entry4.collect(Collections.<Integer>emptyList());
+
+		Thread.sleep(10L);
+
+		Assert.assertEquals(4, queue.size());
+
+		entry1.collect(Collections.<Integer>emptyList());
+
+		Assert.assertEquals(expected, pollOperation.get());
+
+		verify(operatorActions, never()).failOperator(any(Exception.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java
new file mode 100644
index 0000000..c9e59c7
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.streaming.api.operators.async.OperatorActions;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.streaming.api.operators.async.queue.StreamElementQueueTest.StreamElementQueueType.OrderedStreamElementQueueType;
+import static org.apache.flink.streaming.api.operators.async.queue.StreamElementQueueTest.StreamElementQueueType.UnorderedStreamElementQueueType;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for the basic functionality of {@link StreamElementQueue}. The basic operations consist
+ * of putting and polling elements from the queue.
+ */
+@RunWith(Parameterized.class)
+public class StreamElementQueueTest extends TestLogger {
+
+	private static final long timeout = 10000L;
+	private static ExecutorService executor;
+
+	@BeforeClass
+	public static void setup() {
+		executor = Executors.newFixedThreadPool(3);
+	}
+
+	@AfterClass
+	public static void shutdown() {
+		executor.shutdown();
+
+		try {
+			if (!executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)) {
+				executor.shutdownNow();
+			}
+		} catch (InterruptedException interrupted) {
+			executor.shutdownNow();
+
+			Thread.currentThread().interrupt();
+		}
+	}
+
+	enum StreamElementQueueType {
+		OrderedStreamElementQueueType,
+		UnorderedStreamElementQueueType
+	}
+
+	@Parameterized.Parameters
+	public static Collection<StreamElementQueueType> streamElementQueueTypes() {
+		return Arrays.asList(OrderedStreamElementQueueType, UnorderedStreamElementQueueType);
+	}
+
+	private final StreamElementQueueType streamElementQueueType;
+
+	public StreamElementQueueTest(StreamElementQueueType streamElementQueueType) {
+		this.streamElementQueueType = Preconditions.checkNotNull(streamElementQueueType);
+	}
+
+	public StreamElementQueue createStreamElementQueue(int capacity, OperatorActions operatorActions) {
+		switch (streamElementQueueType) {
+			case OrderedStreamElementQueueType:
+				return new OrderedStreamElementQueue(capacity, executor, operatorActions);
+			case UnorderedStreamElementQueueType:
+				return new UnorderedStreamElementQueue(capacity, executor, operatorActions);
+			default:
+				throw new IllegalStateException("Unknown stream element queue type: " + streamElementQueueType);
+		}
+	}
+
+	@Test
+	public void testPut() throws InterruptedException {
+		OperatorActions operatorActions = mock(OperatorActions.class);
+		StreamElementQueue queue = createStreamElementQueue(2, operatorActions);
+
+		final Watermark watermark = new Watermark(0L);
+		final StreamRecord<Integer> streamRecord = new StreamRecord<>(42, 1L);
+		final Watermark nextWatermark = new Watermark(2L);
+
+		final WatermarkQueueEntry watermarkQueueEntry = new WatermarkQueueEntry(watermark);
+		final StreamRecordQueueEntry<Integer> streamRecordQueueEntry = new StreamRecordQueueEntry<>(streamRecord);
+
+		queue.put(watermarkQueueEntry);
+		queue.put(streamRecordQueueEntry);
+
+		Assert.assertEquals(2, queue.size());
+
+		Assert.assertFalse(queue.tryPut(new WatermarkQueueEntry(nextWatermark)));
+
+		Collection<StreamElementQueueEntry<?>> actualValues = queue.values();
+
+		List<StreamElementQueueEntry<?>> expectedValues = Arrays.asList(watermarkQueueEntry, streamRecordQueueEntry);
+
+		Assert.assertEquals(expectedValues, actualValues);
+
+		verify(operatorActions, never()).failOperator(any(Exception.class));
+	}
+
+	@Test
+	public void testPoll() throws InterruptedException {
+		OperatorActions operatorActions = mock(OperatorActions.class);
+		StreamElementQueue queue = createStreamElementQueue(2, operatorActions);
+
+		WatermarkQueueEntry watermarkQueueEntry = new WatermarkQueueEntry(new Watermark(0L));
+		StreamRecordQueueEntry<Integer> streamRecordQueueEntry = new StreamRecordQueueEntry<>(new StreamRecord<>(42, 1L));
+
+		queue.put(watermarkQueueEntry);
+		queue.put(streamRecordQueueEntry);
+
+		Assert.assertEquals(watermarkQueueEntry, queue.peekBlockingly());
+		Assert.assertEquals(2, queue.size());
+
+		Assert.assertEquals(watermarkQueueEntry, queue.poll());
+		Assert.assertEquals(1, queue.size());
+
+		streamRecordQueueEntry.collect(Collections.<Integer>emptyList());
+
+		Assert.assertEquals(streamRecordQueueEntry, queue.poll());
+
+		Assert.assertEquals(0, queue.size());
+		Assert.assertTrue(queue.isEmpty());
+
+		verify(operatorActions, never()).failOperator(any(Exception.class));
+	}
+
+	/**
+	 * Tests that a put operation blocks if the queue is full.
+	 */
+	@Test
+	public void testBlockingPut() throws Exception {
+		OperatorActions operatorActions = mock(OperatorActions.class);
+		final StreamElementQueue queue = createStreamElementQueue(1, operatorActions);
+
+		StreamRecordQueueEntry<Integer> streamRecordQueueEntry = new StreamRecordQueueEntry<>(new StreamRecord<>(42, 0L));
+		final StreamRecordQueueEntry<Integer> streamRecordQueueEntry2 = new StreamRecordQueueEntry<>(new StreamRecord<>(43, 1L));
+
+		queue.put(streamRecordQueueEntry);
+
+		Assert.assertEquals(1, queue.size());
+
+		Future<Void> putOperation = FlinkFuture.supplyAsync(new Callable<Void>() {
+			@Override
+			public Void call() throws Exception {
+				queue.put(streamRecordQueueEntry2);
+
+				return null;
+			}
+		}, executor);
+
+		// give the future a chance to complete
+		Thread.sleep(10L);
+
+		// but it shouldn't ;-)
+		Assert.assertFalse(putOperation.isDone());
+
+		streamRecordQueueEntry.collect(Collections.<Integer>emptyList());
+
+		// polling the completed head element frees the queue again
+		Assert.assertEquals(streamRecordQueueEntry, queue.poll());
+
+		// now the put operation should complete
+		putOperation.get();
+
+		verify(operatorActions, never()).failOperator(any(Exception.class));
+	}
+
+	/**
+	 * Test that a poll operation on an empty queue blocks.
+	 */
+	@Test
+	public void testBlockingPoll() throws Exception {
+		OperatorActions operatorActions = mock(OperatorActions.class);
+		final StreamElementQueue queue = createStreamElementQueue(1, operatorActions);
+
+		WatermarkQueueEntry watermarkQueueEntry = new WatermarkQueueEntry(new Watermark(1L));
+		StreamRecordQueueEntry<Integer> streamRecordQueueEntry = new StreamRecordQueueEntry<>(new StreamRecord<>(1, 2L));
+
+		Assert.assertTrue(queue.isEmpty());
+
+		Future<AsyncResult> peekOperation = FlinkFuture.supplyAsync(new Callable<AsyncResult>() {
+			@Override
+			public AsyncResult call() throws Exception {
+				return queue.peekBlockingly();
+			}
+		}, executor);
+
+		Thread.sleep(10L);
+
+		Assert.assertFalse(peekOperation.isDone());
+
+		queue.put(watermarkQueueEntry);
+
+		AsyncResult watermarkResult = peekOperation.get();
+
+		Assert.assertEquals(watermarkQueueEntry, watermarkResult);
+		Assert.assertEquals(1, queue.size());
+
+		Assert.assertEquals(watermarkQueueEntry, queue.poll());
+		Assert.assertTrue(queue.isEmpty());
+
+		Future<AsyncResult> pollOperation = FlinkFuture.supplyAsync(new Callable<AsyncResult>() {
+			@Override
+			public AsyncResult call() throws Exception {
+				return queue.poll();
+			}
+		}, executor);
+
+		Thread.sleep(10L);
+
+		Assert.assertFalse(pollOperation.isDone());
+
+		queue.put(streamRecordQueueEntry);
+
+		Thread.sleep(10L);
+
+		Assert.assertFalse(pollOperation.isDone());
+
+		streamRecordQueueEntry.collect(Collections.<Integer>emptyList());
+
+		Assert.assertEquals(streamRecordQueueEntry, pollOperation.get());
+
+		Assert.assertTrue(queue.isEmpty());
+
+		verify(operatorActions, never()).failOperator(any(Exception.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java
new file mode 100644
index 0000000..0a57f92
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueueTest.java
@@ -0,0 +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.async.queue;
+
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.streaming.api.operators.async.OperatorActions;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+
+/**
+ * {@link UnorderedStreamElementQueue} specific tests
+ */
+public class UnorderedStreamElementQueueTest extends TestLogger {
+	private static final long timeout = 10000L;
+	private static ExecutorService executor;
+
+	@BeforeClass
+	public static void setup() {
+		executor = Executors.newFixedThreadPool(3);
+	}
+
+	@AfterClass
+	public static void shutdown() {
+		executor.shutdown();
+
+		try {
+			if (!executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)) {
+				executor.shutdownNow();
+			}
+		} catch (InterruptedException interrupted) {
+			executor.shutdownNow();
+
+			Thread.currentThread().interrupt();
+		}
+	}
+
+	/**
+	 * Tests that only elements before the oldest watermark are returned if they are completed.
+	 */
+	@Test
+	public void testCompletionOrder() throws Exception {
+		OperatorActions operatorActions = mock(OperatorActions.class);
+
+		final UnorderedStreamElementQueue queue = new UnorderedStreamElementQueue(8, executor, operatorActions);
+
+		StreamRecordQueueEntry<Integer> record1 = new StreamRecordQueueEntry<>(new StreamRecord<>(1, 0L));
+		StreamRecordQueueEntry<Integer> record2 = new StreamRecordQueueEntry<>(new StreamRecord<>(2, 1L));
+		WatermarkQueueEntry watermark1 = new WatermarkQueueEntry(new Watermark(2L));
+		StreamRecordQueueEntry<Integer> record3 = new StreamRecordQueueEntry<>(new StreamRecord<>(3, 3L));
+		StreamRecordQueueEntry<Integer> record4 = new StreamRecordQueueEntry<>(new StreamRecord<>(4, 4L));
+		WatermarkQueueEntry watermark2 = new WatermarkQueueEntry(new Watermark(5L));
+		StreamRecordQueueEntry<Integer> record5 = new StreamRecordQueueEntry<>(new StreamRecord<>(5, 6L));
+		StreamRecordQueueEntry<Integer> record6 = new StreamRecordQueueEntry<>(new StreamRecord<>(6, 7L));
+
+		List<StreamElementQueueEntry<?>> entries = Arrays.asList(record1, record2, watermark1, record3,
+			record4, watermark2, record5, record6);
+
+		// The queue should look like R1, R2, W1, R3, R4, W2, R5, R6
+		for (StreamElementQueueEntry<?> entry : entries) {
+			queue.put(entry);
+		}
+
+		Assert.assertTrue(8 == queue.size());
+
+		Future<AsyncResult> firstPoll = FlinkFuture.supplyAsync(new Callable<AsyncResult>() {
+			@Override
+			public AsyncResult call() throws Exception {
+				return queue.poll();
+			}
+		}, executor);
+
+		// this should not fulfill the poll, because R3 is behind W1
+		record3.collect(Collections.<Integer>emptyList());
+
+		Thread.sleep(10L);
+
+		Assert.assertFalse(firstPoll.isDone());
+
+		record2.collect(Collections.<Integer>emptyList());
+
+		Assert.assertEquals(record2, firstPoll.get());
+
+		Future<AsyncResult> secondPoll = FlinkFuture.supplyAsync(new Callable<AsyncResult>() {
+			@Override
+			public AsyncResult call() throws Exception {
+				return queue.poll();
+			}
+		}, executor);
+
+		record6.collect(Collections.<Integer>emptyList());
+		record4.collect(Collections.<Integer>emptyList());
+
+		Thread.sleep(10L);
+
+		// The future should not be completed because R1 has not been completed yet
+		Assert.assertFalse(secondPoll.isDone());
+
+		record1.collect(Collections.<Integer>emptyList());
+
+		Assert.assertEquals(record1, secondPoll.get());
+
+		// Now W1, R3, R4 and W2 are completed and should be pollable
+		Assert.assertEquals(watermark1, queue.poll());
+
+		// The order of R3 and R4 is not specified
+		Set<AsyncResult> expected = new HashSet<>(2);
+		expected.add(record3);
+		expected.add(record4);
+
+		Set<AsyncResult> actual = new HashSet<>(2);
+
+		actual.add(queue.poll());
+		actual.add(queue.poll());
+
+		Assert.assertEquals(expected, actual);
+
+		Assert.assertEquals(watermark2, queue.poll());
+
+		// since R6 has been completed before and W2 has been consumed, we should be able to poll
+		// this record as well
+		Assert.assertEquals(record6, queue.poll());
+
+		// only R5 left in the queue
+		Assert.assertTrue(1 == queue.size());
+
+		Future<AsyncResult> thirdPoll = FlinkFuture.supplyAsync(new Callable<AsyncResult>() {
+			@Override
+			public AsyncResult call() throws Exception {
+				return queue.poll();
+			}
+		}, executor);
+
+		Thread.sleep(10L);
+
+		Assert.assertFalse(thirdPoll.isDone());
+
+		record5.collect(Collections.<Integer>emptyList());
+
+		Assert.assertEquals(record5, thirdPoll.get());
+
+		Assert.assertTrue(queue.isEmpty());
+
+		verify(operatorActions, never()).failOperator(any(Exception.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
index f87b5ef..e600420 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
@@ -40,6 +40,7 @@ import org.apache.flink.streaming.runtime.tasks.StreamTask;
 
 import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
 import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.util.CollectorOutput;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
@@ -340,33 +341,4 @@ public class StreamSourceOperatorTest {
 			running = false;
 		}
 	}
-
-	// ------------------------------------------------------------------------
-	
-	private static class CollectorOutput<T> implements Output<StreamRecord<T>> {
-
-		private final List<StreamElement> list;
-
-		private CollectorOutput(List<StreamElement> list) {
-			this.list = list;
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			list.add(mark);
-		}
-
-		@Override
-		public void emitLatencyMarker(LatencyMarker latencyMarker) {
-			list.add(latencyMarker);
-		}
-
-		@Override
-		public void collect(StreamRecord<T> record) {
-			list.add(record);
-		}
-
-		@Override
-		public void close() {}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/CollectorOutput.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/CollectorOutput.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/CollectorOutput.java
new file mode 100644
index 0000000..fcc8a6c
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/CollectorOutput.java
@@ -0,0 +1,57 @@
+/*
+ * 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.commons.lang3.SerializationUtils;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.io.Serializable;
+import java.util.List;
+
+public class CollectorOutput<T> implements Output<StreamRecord<T>> {
+
+	private final List<StreamElement> list;
+
+	public CollectorOutput(List<StreamElement> list) {
+		this.list = list;
+	}
+
+	@Override
+	public void emitWatermark(Watermark mark) {
+		list.add(mark);
+	}
+
+	@Override
+	public void emitLatencyMarker(LatencyMarker latencyMarker) {
+		list.add(latencyMarker);
+	}
+
+	@Override
+	public void collect(StreamRecord<T> record) {
+		T copied = SerializationUtils.deserialize(SerializationUtils.serialize((Serializable) record.getValue()));
+		list.add(record.copy(copied));
+	}
+
+	@Override
+	public void close() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
index ea99fe3..3631965 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
@@ -38,6 +38,7 @@ import org.apache.flink.util.MathUtils;
 import org.junit.*;
 
 import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
@@ -94,6 +95,9 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		final MemorySinkFunction sinkFunction1 = new MemorySinkFunction(0);
 
+		final List<Integer> actualResult1 = new ArrayList<>();
+		MemorySinkFunction.registerCollection(0, actualResult1);
+
 		splittedResult.select("0").map(new MapFunction<Tuple2<Integer,Integer>, Integer>() {
 			private static final long serialVersionUID = 2114608668010092995L;
 
@@ -103,9 +107,11 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 			}
 		}).addSink(sinkFunction1);
 
-
 		final MemorySinkFunction sinkFunction2 = new MemorySinkFunction(1);
 
+		final List<Integer> actualResult2 = new ArrayList<>();
+		MemorySinkFunction.registerCollection(1, actualResult2);
+
 		splittedResult.select("1").map(new MapFunction<Tuple2<Integer, Integer>, Integer>() {
 			private static final long serialVersionUID = 5631104389744681308L;
 
@@ -132,13 +138,11 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		env.execute();
 
-		Collection<Integer> result1 = sinkFunction1.getResult();
-		Collections.sort((ArrayList)result1);
-		Collection<Integer> result2 = sinkFunction2.getResult();
-		Collections.sort((ArrayList)result2);
+		Collections.sort(actualResult1);
+		Collections.sort(actualResult2);
 
-		Assert.assertArrayEquals(result1.toArray(), expected1.toArray());
-		Assert.assertArrayEquals(result2.toArray(), expected2.toArray());
+		Assert.assertEquals(expected1, actualResult1);
+		Assert.assertEquals(expected2, actualResult2);
 
 		MemorySinkFunction.clear();
 	}
@@ -155,6 +159,8 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<Integer, NonSerializable>> input = env.addSource(new NonSerializableTupleSource(numElements));
 
 		final MemorySinkFunction sinkFunction = new MemorySinkFunction(0);
+		final ArrayList<Integer> actualResult = new ArrayList<>();
+		MemorySinkFunction.registerCollection(0, actualResult);
 
 		input
 			.keyBy(0)
@@ -186,23 +192,28 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		env.execute();
 
-		Collection<Integer> result = sinkFunction.getResult();
-		Collections.sort((ArrayList)result);
+		Collections.sort(actualResult);
 
-		Assert.assertArrayEquals(result.toArray(), expected.toArray());
+		Assert.assertEquals(expected, actualResult);
 
 		MemorySinkFunction.clear();
 	}
 
+	/**
+	 * Tests the basic functionality of the AsyncWaitOperator: Processing a limited stream of
+	 * elements by doubling their value. This is tested in for the ordered and unordered mode.
+	 */
 	@Test
 	public void testAsyncWaitOperator() throws Exception {
-		final int numElements = 10;
+		final int numElements = 5;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStream<Tuple2<Integer, NonSerializable>> input = env.addSource(new NonSerializableTupleSource(numElements));
 
 		AsyncFunction<Tuple2<Integer, NonSerializable>, Integer> function = new RichAsyncFunction<Tuple2<Integer, NonSerializable>, Integer>() {
+			private static final long serialVersionUID = 7000343199829487985L;
+
 			transient ExecutorService executorService;
 
 			@Override
@@ -214,26 +225,16 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 			@Override
 			public void close() throws Exception {
 				super.close();
-				executorService.shutdown();
+				executorService.shutdownNow();
 			}
 
 			@Override
 			public void asyncInvoke(final Tuple2<Integer, NonSerializable> input,
 									final AsyncCollector<Integer> collector) throws Exception {
-				this.executorService.submit(new Runnable() {
+				executorService.submit(new Runnable() {
 					@Override
 					public void run() {
-						// wait for while to simulate async operation here
-						int sleep = (int) (new Random().nextFloat() * 10);
-						try {
-							Thread.sleep(sleep);
-							List<Integer> ret = new ArrayList<>();
-							ret.add(input.f0+input.f0);
-							collector.collect(ret);
-						}
-						catch (InterruptedException e) {
-							collector.collect(new ArrayList<Integer>(0));
-						}
+						collector.collect(Collections.singletonList(input.f0 + input.f0));
 					}
 				});
 			}
@@ -243,6 +244,8 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		// save result from ordered process
 		final MemorySinkFunction sinkFunction1 = new MemorySinkFunction(0);
+		final List<Integer> actualResult1 = new ArrayList<>(numElements);
+		MemorySinkFunction.registerCollection(0, actualResult1);
 
 		orderedResult.addSink(sinkFunction1).setParallelism(1);
 
@@ -251,6 +254,8 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		// save result from unordered process
 		final MemorySinkFunction sinkFunction2 = new MemorySinkFunction(1);
+		final List<Integer> actualResult2 = new ArrayList<>(numElements);
+		MemorySinkFunction.registerCollection(1, actualResult2);
 
 		unorderedResult.addSink(sinkFunction2);
 
@@ -263,11 +268,10 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		env.execute();
 
-		Assert.assertArrayEquals(expected.toArray(), sinkFunction1.getResult().toArray());
+		Assert.assertEquals(expected, actualResult1);
 
-		Collection<Integer> result = sinkFunction2.getResult();
-		Collections.sort((ArrayList)result);
-		Assert.assertArrayEquals(expected.toArray(), result.toArray());
+		Collections.sort(actualResult2);
+		Assert.assertEquals(expected, actualResult2);
 
 		MemorySinkFunction.clear();
 	}
@@ -331,43 +335,31 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 	}
 
 	private static class MemorySinkFunction implements SinkFunction<Integer> {
-		private final static Collection<Integer> collection1 = new ArrayList<>(10);
+		private static Map<Integer, Collection<Integer>> collections = new ConcurrentHashMap<>();
 
-		private final static Collection<Integer> collection2 = new ArrayList<>(10);
+		private static final long serialVersionUID = -8815570195074103860L;
 
-		private  final long serialVersionUID = -8815570195074103860L;
+		private final int key;
 
-		private final int idx;
-
-		public MemorySinkFunction(int idx) {
-			this.idx = idx;
+		public MemorySinkFunction(int key) {
+			this.key = key;
 		}
 
 		@Override
 		public void invoke(Integer value) throws Exception {
-			if (idx == 0) {
-				synchronized (collection1) {
-					collection1.add(value);
-				}
-			}
-			else {
-				synchronized (collection2) {
-					collection2.add(value);
-				}
-			}
-		}
+			Collection<Integer> collection = collections.get(key);
 
-		public Collection<Integer> getResult() {
-			if (idx == 0) {
-				return collection1;
+			synchronized (collection) {
+				collection.add(value);
 			}
+		}
 
-			return collection2;
+		public static void registerCollection(int key, Collection<Integer> collection) {
+			collections.put(key, collection);
 		}
 
 		public static void clear() {
-			collection1.clear();
-			collection2.clear();
+			collections.clear();
 		}
 	}
 }


[4/7] flink git commit: [FLINK-4391] Polish asynchronous I/O operations

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java
new file mode 100644
index 0000000..5a2e43c
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/OperatorActions.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.operators.async;
+
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+/**
+ * Interface for {@link StreamOperator} actions.
+ */
+public interface OperatorActions {
+
+	/**
+	 * Fail the respective stream operator with the given throwable.
+	 *
+	 * @param throwable to fail the stream operator with
+	 */
+	void failOperator(Throwable throwable);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java
new file mode 100644
index 0000000..8088bf0
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncCollectionResult.java
@@ -0,0 +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
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.operators.async.queue;
+
+import java.util.Collection;
+
+/**
+ * {@link AsyncResult} sub class for asynchronous result collections.
+ *
+ * @param <T> Type of the collection elements.
+ */
+public interface AsyncCollectionResult<T> extends AsyncResult {
+
+	boolean hasTimestamp();
+
+	long getTimestamp();
+
+	/**
+	 * Return the asynchronous result collection.
+	 *
+	 * @return the asynchronous result collection
+	 * @throws Exception if the asynchronous result collection could not be completed
+	 */
+	Collection<T> get() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java
new file mode 100644
index 0000000..1a99928
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncResult.java
@@ -0,0 +1,59 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * Asynchronous result returned by the {@link StreamElementQueue}. The asynchronous result can
+ * either be a {@link Watermark} or a collection of new output elements produced by the
+ * {@link AsyncFunction}.
+ */
+public interface AsyncResult {
+
+	/**
+	 * True if the async result is a {@link Watermark}; otherwise false.
+	 *
+	 * @return True if the async result is a {@link Watermark}; otherwise false.
+	 */
+	boolean isWatermark();
+
+	/**
+	 * True fi the async result is a collection of output elements; otherwise false.
+	 *
+	 * @return True if the async reuslt is a collection of output elements; otherwise false
+	 */
+	boolean isResultCollection();
+
+	/**
+	 * Return this async result as a async watermark result.
+	 *
+	 * @return this result as a {@link AsyncWatermarkResult}.
+	 */
+	AsyncWatermarkResult asWatermark();
+
+	/**
+	 * Return this async result as a async result collection.
+	 *
+	 * @param <T> Type of the result collection's elements
+	 * @return this result as a {@link AsyncCollectionResult}.
+	 */
+	<T> AsyncCollectionResult<T> asResultCollection();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java
new file mode 100644
index 0000000..c19b520
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/AsyncWatermarkResult.java
@@ -0,0 +1,33 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * {@link AsyncResult} subclass for asynchronous result {@link Watermark}.
+ */
+public interface AsyncWatermarkResult extends AsyncResult {
+	/**
+	 * Get the resulting watermark.
+	 *
+	 * @return the asynchronous result watermark
+	 */
+	Watermark getWatermark();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java
new file mode 100644
index 0000000..2bbcb6c
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/OrderedStreamElementQueue.java
@@ -0,0 +1,229 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.streaming.api.operators.async.OperatorActions;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.Executor;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Ordered {@link StreamElementQueue} implementation. The ordered stream element queue emits
+ * asynchronous results in the order in which the {@link StreamElementQueueEntry} have been added
+ * to the queue. Thus, even if the completion order can be arbitrary, the output order strictly
+ * follows the insertion order (element cannot overtake each other).
+ */
+public class OrderedStreamElementQueue implements StreamElementQueue {
+
+	private static final Logger LOG = LoggerFactory.getLogger(OrderedStreamElementQueue.class);
+
+	/** Capacity of this queue */
+	private final int capacity;
+
+	/** Executor to run the onCompletion callback */
+	private final Executor executor;
+
+	/** Operator actions to signal a failure to the operator */
+	private final OperatorActions operatorActions;
+
+	/** Lock and conditions for the blocking queue */
+	private final ReentrantLock lock;
+	private final Condition notFull;
+	private final Condition headIsCompleted;
+
+	/** Queue for the inserted StreamElementQueueEntries */
+	private final ArrayDeque<StreamElementQueueEntry<?>> queue;
+
+	public OrderedStreamElementQueue(
+			int capacity,
+			Executor executor,
+			OperatorActions operatorActions) {
+
+		Preconditions.checkArgument(capacity > 0, "The capacity must be larger than 0.");
+		this.capacity = capacity;
+
+		this.executor = Preconditions.checkNotNull(executor, "executor");
+
+		this.operatorActions = Preconditions.checkNotNull(operatorActions, "operatorActions");
+
+		this.lock = new ReentrantLock(false);
+		this.headIsCompleted = lock.newCondition();
+		this.notFull = lock.newCondition();
+
+		this.queue = new ArrayDeque<>(capacity);
+	}
+
+	@Override
+	public AsyncResult peekBlockingly() throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			while (queue.isEmpty() || !queue.peek().isDone()) {
+				headIsCompleted.await();
+			}
+
+			LOG.debug("Peeked head element from ordered stream element queue with filling degree " +
+				"({}/{}).", queue.size(), capacity);
+
+			return queue.peek();
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public AsyncResult poll() throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			while (queue.isEmpty() || !queue.peek().isDone()) {
+				headIsCompleted.await();
+			}
+
+			notFull.signalAll();
+
+			LOG.debug("Polled head element from ordered stream element queue. New filling degree " +
+				"({}/{}).", queue.size() - 1, capacity);
+
+			return queue.poll();
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public Collection<StreamElementQueueEntry<?>> values() throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			StreamElementQueueEntry<?>[] array = new StreamElementQueueEntry[queue.size()];
+
+			array = queue.toArray(array);
+
+			return Arrays.asList(array);
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public boolean isEmpty() {
+		return queue.isEmpty();
+	}
+
+	@Override
+	public int size() {
+		return queue.size();
+	}
+
+	@Override
+	public <T> void put(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			while (queue.size() >= capacity) {
+				notFull.await();
+			}
+
+			addEntry(streamElementQueueEntry);
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public <T> boolean tryPut(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			if (queue.size() < capacity) {
+				addEntry(streamElementQueueEntry);
+
+				LOG.debug("Put element into ordered stream element queue. New filling degree " +
+					"({}/{}).", queue.size(), capacity);
+
+				return true;
+			} else {
+				LOG.debug("Failed to put element into ordered stream element queue because it " +
+					"was full ({}/{}).", queue.size(), capacity);
+
+				return false;
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Add the given {@link StreamElementQueueEntry} to the queue. Additionally, this method
+	 * registers a onComplete callback which is triggered once the given queue entry is completed.
+	 *
+	 * @param streamElementQueueEntry to be inserted
+	 * @param <T> Type of the stream element queue entry's result
+	 */
+	private <T> void addEntry(StreamElementQueueEntry<T> streamElementQueueEntry) {
+		assert(lock.isHeldByCurrentThread());
+
+		queue.addLast(streamElementQueueEntry);
+
+		streamElementQueueEntry.onComplete(new AcceptFunction<StreamElementQueueEntry<T>>() {
+			@Override
+			public void accept(StreamElementQueueEntry<T> value) {
+				try {
+					onCompleteHandler(value);
+				} catch (InterruptedException e) {
+					// we got interrupted. This indicates a shutdown of the executor
+					LOG.debug("AsyncBufferEntry could not be properly completed because the " +
+						"executor thread has been interrupted.", e);
+				} catch (Throwable t) {
+					operatorActions.failOperator(new Exception("Could not complete the " +
+						"stream element queue entry: " + value + '.', t));
+				}
+			}
+		}, executor);
+	}
+
+	/**
+	 * Check if the completed {@link StreamElementQueueEntry} is the current head. If this is the
+	 * case, then notify the consumer thread about a new consumable entry.
+	 *
+	 * @param streamElementQueueEntry which has been completed
+	 * @throws InterruptedException if the current thread is interrupted
+	 */
+	private void onCompleteHandler(StreamElementQueueEntry<?> streamElementQueueEntry) throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			if (!queue.isEmpty() && queue.peek().isDone()) {
+				LOG.debug("Signal ordered stream element queue has completed head element.");
+				headIsCompleted.signalAll();
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java
new file mode 100644
index 0000000..1a2c4a8
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueue.java
@@ -0,0 +1,96 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator;
+
+import java.util.Collection;
+
+/**
+ * Interface for blocking stream element queues for the {@link AsyncWaitOperator}.
+ */
+public interface StreamElementQueue {
+
+	/**
+	 * Put the given element in the queue if capacity is left. If not, then block until this is
+	 * the case.
+	 *
+	 * @param streamElementQueueEntry to be put into the queue
+	 * @param <T> Type of the entries future value
+	 * @throws InterruptedException if the calling thread has been interrupted while waiting to
+	 * 	insert the given element
+	 */
+	<T> void put(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException;
+
+	/**
+	 * Try to put the given element in the queue. This operation succeeds if the queue has capacity
+	 * left and fails if the queue is full.
+	 *
+	 * @param streamElementQueueEntry to be inserted
+	 * @param <T> Type of the entries future value
+	 * @return True if the entry could be inserted; otherwise false
+	 * @throws InterruptedException if the calling thread has been interrupted while waiting to
+	 * 	insert the given element
+	 */
+	<T> boolean tryPut(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException;
+
+	/**
+	 * Peek at the head of the queue and return the first completed {@link AsyncResult}. This
+	 * operation is a blocking operation and only returns once a completed async result has been
+	 * found.
+	 *
+	 * @return Completed {@link AsyncResult}
+	 * @throws InterruptedException if the current thread has been interrupted while waiting for a
+	 * 	completed async result.
+	 */
+	AsyncResult peekBlockingly() throws InterruptedException;
+
+	/**
+	 * Poll the first completed {@link AsyncResult} from the head of this queue. This operation is
+	 * blocking and only returns once a completed async result has been found.
+	 *
+	 * @return Completed {@link AsyncResult} which has been removed from the queue
+	 * @throws InterruptedException if the current thread has been interrupted while waiting for a
+	 * 	completed async result.
+	 */
+	AsyncResult poll() throws InterruptedException;
+
+	/**
+	 * Return the collection of {@link StreamElementQueueEntry} currently contained in this queue.
+	 *
+	 * @return Collection of currently contained {@link StreamElementQueueEntry}.
+	 * @throws InterruptedException if the current thread has been interrupted while retrieving the
+	 * 	stream element queue entries of this queue.
+	 */
+	Collection<StreamElementQueueEntry<?>> values() throws InterruptedException;
+
+	/**
+	 * True if the queue is empty; otherwise false.
+	 *
+	 * @return True if the queue is empty; otherwise false.
+	 */
+	boolean isEmpty();
+
+	/**
+	 * Return the size of the queue.
+	 *
+	 * @return The number of elements contained in this queue.
+	 */
+	int size();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
new file mode 100644
index 0000000..06ebf3c
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
@@ -0,0 +1,97 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.util.Preconditions;
+
+import java.util.concurrent.Executor;
+
+/**
+ * Entry class for the {@link StreamElementQueue}. The stream element queue entry stores the
+ * {@link StreamElement} for which the stream element queue entry has been instantiated.
+ * Furthermore, it allows to register callbacks for when the queue entry is completed.
+ *
+ * @param <T> Type of the result
+ */
+public abstract class StreamElementQueueEntry<T> implements AsyncResult {
+
+	/** Stream element */
+	private final StreamElement streamElement;
+
+	public StreamElementQueueEntry(StreamElement streamElement) {
+		this.streamElement = Preconditions.checkNotNull(streamElement);
+	}
+
+	public StreamElement getStreamElement() {
+		return streamElement;
+	}
+
+	/**
+	 * True if the stream element queue entry has been completed; otherwise false.
+	 *
+	 * @return True if the stream element queue entry has been completed; otherwise false.
+	 */
+	public boolean isDone() {
+		return getFuture().isDone();
+	}
+
+	/**
+	 * Register the given complete function to be called once this queue entry has been completed.
+	 *
+	 * @param completeFunction to call when the queue entry has been completed
+	 * @param executor to run the complete function
+	 */
+	public void onComplete(
+			final AcceptFunction<StreamElementQueueEntry<T>> completeFunction,
+			Executor executor) {
+		final StreamElementQueueEntry<T> thisReference = this;
+
+		getFuture().thenAcceptAsync(new AcceptFunction<T>() {
+			@Override
+			public void accept(T value) {
+				completeFunction.accept(thisReference);
+			}
+		}, executor);
+	}
+
+	protected abstract Future<T> getFuture();
+
+	@Override
+	public final boolean isWatermark() {
+		return AsyncWatermarkResult.class.isAssignableFrom(getClass());
+	}
+
+	@Override
+	public final boolean isResultCollection() {
+		return AsyncCollectionResult.class.isAssignableFrom(getClass());
+	}
+
+	@Override
+	public final AsyncWatermarkResult asWatermark() {
+		return (AsyncWatermarkResult) this;
+	}
+
+	@Override
+	public final <T> AsyncCollectionResult<T> asResultCollection() {
+		return (AsyncCollectionResult<T>) this;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java
new file mode 100644
index 0000000..f0e707e
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamRecordQueueEntry.java
@@ -0,0 +1,85 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.runtime.concurrent.CompletableFuture;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.util.Collection;
+
+/**
+ * {@link StreamElementQueueEntry} implementation for {@link StreamRecord}. This class also acts
+ * as the {@link AsyncCollector} implementation which is given to the {@link AsyncFunction}. The
+ * async function completes this class with a collection of results.
+ *
+ * @param <OUT> Type of the asynchronous collection result
+ */
+public class StreamRecordQueueEntry<OUT> extends StreamElementQueueEntry<Collection<OUT>>
+	implements AsyncCollectionResult<OUT>, AsyncCollector<OUT> {
+
+	/** Timestamp information */
+	private final boolean hasTimestamp;
+	private final long timestamp;
+
+	/** Future containing the collection result */
+	private final CompletableFuture<Collection<OUT>> resultFuture;
+
+	public StreamRecordQueueEntry(StreamRecord<?> streamRecord) {
+		super(streamRecord);
+
+		hasTimestamp = streamRecord.hasTimestamp();
+		timestamp = streamRecord.getTimestamp();
+
+		resultFuture = new FlinkCompletableFuture<>();
+	}
+
+	@Override
+	public boolean hasTimestamp() {
+		return hasTimestamp;
+	}
+
+	@Override
+	public long getTimestamp() {
+		return timestamp;
+	}
+
+	@Override
+	public Collection<OUT> get() throws Exception {
+		return resultFuture.get();
+	}
+
+	@Override
+	protected Future<Collection<OUT>> getFuture() {
+		return resultFuture;
+	}
+
+	@Override
+	public void collect(Collection<OUT> result) {
+		resultFuture.complete(result);
+	}
+
+	@Override
+	public void collect(Throwable error) {
+		resultFuture.completeExceptionally(error);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
new file mode 100644
index 0000000..603d8cc
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
@@ -0,0 +1,304 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.streaming.api.operators.async.OperatorActions;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Unordered implementation of the {@link StreamElementQueue}. The unordered stream element queue
+ * emits asynchronous results as soon as they are completed. Additionally it maintains the
+ * watermark-stream record order. This means that no stream record can be overtaken by a watermark
+ * and no watermark can overtake a stream record. However, stream records falling in the same
+ * segment between two watermarks can overtake each other (their emission order is not guaranteed).
+ */
+public class UnorderedStreamElementQueue implements StreamElementQueue {
+
+	private static final Logger LOG = LoggerFactory.getLogger(UnorderedStreamElementQueue.class);
+
+	/** Capacity of this queue */
+	private final int capacity;
+
+	/** Executor to run the onComplete callbacks */
+	private final Executor executor;
+
+	/** OperatorActions to signal the owning operator a failure */
+	private final OperatorActions operatorActions;
+
+	/** Queue of uncompleted stream element queue entries segmented by watermarks */
+	private final ArrayDeque<Set<StreamElementQueueEntry<?>>> uncompletedQueue;
+
+	/** Queue of completed stream element queue entries */
+	private final ArrayDeque<StreamElementQueueEntry<?>> completedQueue;
+
+	/** First (chronologically oldest) uncompleted set of stream element queue entries */
+	private Set<StreamElementQueueEntry<?>> firstSet;
+
+	// Last (chronologically youngest) uncompleted set of stream element queue entries. New
+	// stream element queue entries are inserted into this set.
+	private Set<StreamElementQueueEntry<?>> lastSet;
+	private volatile int numberEntries;
+
+	/** Locks and conditions for the blocking queue */
+	private final ReentrantLock lock;
+	private final Condition notFull;
+	private final Condition hasCompletedEntries;
+
+	public UnorderedStreamElementQueue(
+			int capacity,
+			Executor executor,
+			OperatorActions operatorActions) {
+
+		Preconditions.checkArgument(capacity > 0, "The capacity must be larger than 0.");
+		this.capacity = capacity;
+
+		this.executor = Preconditions.checkNotNull(executor, "executor");
+
+		this.operatorActions = Preconditions.checkNotNull(operatorActions, "operatorActions");
+
+		this.uncompletedQueue = new ArrayDeque<>(capacity);
+		this.completedQueue = new ArrayDeque<>(capacity);
+
+		this.firstSet = new HashSet<>(capacity);
+		this.lastSet = firstSet;
+
+		this.numberEntries = 0;
+
+		this.lock = new ReentrantLock();
+		this.notFull = lock.newCondition();
+		this.hasCompletedEntries = lock.newCondition();
+	}
+
+	@Override
+	public <T> void put(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			while (numberEntries >= capacity) {
+				notFull.await();
+			}
+
+			addEntry(streamElementQueueEntry);
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public <T> boolean tryPut(StreamElementQueueEntry<T> streamElementQueueEntry) throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			if (numberEntries < capacity) {
+				addEntry(streamElementQueueEntry);
+
+				LOG.debug("Put element into ordered stream element queue. New filling degree " +
+					"({}/{}).", numberEntries, capacity);
+
+				return true;
+			} else {
+				LOG.debug("Failed to put element into ordered stream element queue because it " +
+					"was full ({}/{}).", numberEntries, capacity);
+
+				return false;
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public AsyncResult peekBlockingly() throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			while (completedQueue.isEmpty()) {
+				hasCompletedEntries.await();
+			}
+
+			LOG.debug("Peeked head element from ordered stream element queue with filling degree " +
+				"({}/{}).", numberEntries, capacity);
+
+			return completedQueue.peek();
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public AsyncResult poll() throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			while (completedQueue.isEmpty()) {
+				hasCompletedEntries.await();
+			}
+
+			numberEntries--;
+			notFull.signalAll();
+
+			LOG.debug("Polled element from unordered stream element queue. New filling degree " +
+				"({}/{}).", numberEntries, capacity);
+
+			return completedQueue.poll();
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public Collection<StreamElementQueueEntry<?>> values() throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			StreamElementQueueEntry<?>[] array = new StreamElementQueueEntry[numberEntries];
+
+			array = completedQueue.toArray(array);
+
+			int counter = completedQueue.size();
+
+			for (StreamElementQueueEntry<?> entry: firstSet) {
+				array[counter] = entry;
+				counter++;
+			}
+
+			for (Set<StreamElementQueueEntry<?>> asyncBufferEntries : uncompletedQueue) {
+
+				for (StreamElementQueueEntry<?> streamElementQueueEntry : asyncBufferEntries) {
+					array[counter] = streamElementQueueEntry;
+					counter++;
+				}
+			}
+
+			return Arrays.asList(array);
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public boolean isEmpty() {
+		return numberEntries == 0;
+	}
+
+	@Override
+	public int size() {
+		return numberEntries;
+	}
+
+	/**
+	 * Callback for onComplete events for the given stream element queue entry. Whenever a queue
+	 * entry is completed, it is checked whether this entry belogns to the first set. If this is the
+	 * case, then the element is added to the completed entries queue from where it can be consumed.
+	 * If the first set becomes empty, then the next set is polled from the uncompleted entries
+	 * queue. Completed entries from this new set are then added to the completed entries queue.
+	 *
+	 * @param streamElementQueueEntry which has been completed
+	 * @throws InterruptedException if the current thread has been interrupted while performing the
+	 * 	on complete callback.
+	 */
+	public void onCompleteHandler(StreamElementQueueEntry<?> streamElementQueueEntry) throws InterruptedException {
+		lock.lockInterruptibly();
+
+		try {
+			if (firstSet.remove(streamElementQueueEntry)) {
+				completedQueue.offer(streamElementQueueEntry);
+
+				while (firstSet.isEmpty() && firstSet != lastSet) {
+					firstSet = uncompletedQueue.poll();
+
+					Iterator<StreamElementQueueEntry<?>> it = firstSet.iterator();
+
+					while (it.hasNext()) {
+						StreamElementQueueEntry<?> bufferEntry = it.next();
+
+						if (bufferEntry.isDone()) {
+							completedQueue.offer(bufferEntry);
+							it.remove();
+						}
+					}
+				}
+
+				LOG.debug("Signal unordered stream element queue has completed entries.");
+				hasCompletedEntries.signalAll();
+			}
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	/**
+	 * Add the given stream element queue entry to the current last set if it is not a watermark.
+	 * If it is a watermark, then stop adding to the current last set, insert the watermark into its
+	 * own set and add a new last set.
+	 *
+	 * @param streamElementQueueEntry to be inserted
+	 * @param <T> Type of the stream element queue entry's result
+	 */
+	private <T> void addEntry(StreamElementQueueEntry<T> streamElementQueueEntry) {
+		assert(lock.isHeldByCurrentThread());
+
+		if (streamElementQueueEntry.isWatermark()) {
+			lastSet = new HashSet<>(capacity);
+
+			if (firstSet.isEmpty()) {
+				firstSet.add(streamElementQueueEntry);
+			} else {
+				Set<StreamElementQueueEntry<?>> watermarkSet = new HashSet<>(1);
+				watermarkSet.add(streamElementQueueEntry);
+				uncompletedQueue.offer(watermarkSet);
+			}
+			uncompletedQueue.offer(lastSet);
+		} else {
+			lastSet.add(streamElementQueueEntry);
+		}
+
+		streamElementQueueEntry.onComplete(new AcceptFunction<StreamElementQueueEntry<T>>() {
+			@Override
+			public void accept(StreamElementQueueEntry<T> value) {
+				try {
+					onCompleteHandler(value);
+				} catch (InterruptedException e) {
+					// The accept executor thread got interrupted. This is probably cause by
+					// the shutdown of the executor.
+					LOG.debug("AsyncBufferEntry could not be properly completed because the " +
+						"executor thread has been interrupted.", e);
+				} catch (Throwable t) {
+					operatorActions.failOperator(new Exception("Could not complete the " +
+						"stream element queue entry: " + value + '.', t));
+				}
+			}
+		}, executor);
+
+		numberEntries++;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java
new file mode 100644
index 0000000..6fe4f44
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/WatermarkQueueEntry.java
@@ -0,0 +1,47 @@
+/*
+ * 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.async.queue;
+
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * {@link StreamElementQueueEntry} implementation for the {@link Watermark}.
+ */
+public class WatermarkQueueEntry extends StreamElementQueueEntry<Watermark> implements AsyncWatermarkResult {
+
+	private final Future<Watermark> future;
+
+	public WatermarkQueueEntry(Watermark watermark) {
+		super(watermark);
+
+		this.future = FlinkCompletableFuture.completed(watermark);
+	}
+
+	@Override
+	public Watermark getWatermark() {
+		return (Watermark) getStreamElement();
+	}
+
+	@Override
+	protected Future<Watermark> getFuture() {
+		return future;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
index 680cc29..7771064 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
@@ -52,7 +52,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
-
 /**
  * The {@code OperatorChain} contains all operators that are executed as one chain within a single
  * {@link StreamTask}.

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 0fb22b8..bd9215a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -326,11 +326,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 				LOG.error("Could not shut down async checkpoint threads", t);
 			}
 
-			// release the output resources. this method should never fail.
-			if (operatorChain != null) {
-				operatorChain.releaseOutputs();
-			}
-
 			// we must! perform this cleanup
 			try {
 				cleanup();
@@ -344,6 +339,11 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 			if (!disposed) {
 				disposeAllOperators();
 			}
+
+			// release the output resources. this method should never fail.
+			if (operatorChain != null) {
+				operatorChain.releaseOutputs();
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java
index b8788c6..12ac693 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunctionTest.java
@@ -18,147 +18,252 @@
 
 package org.apache.flink.streaming.api.functions.async;
 
-import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
 import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
-import org.junit.Assert;
 import org.junit.Test;
 
-import static org.mockito.Matchers.anyString;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 /**
- * Test case for {@link RichAsyncFunction}
+ * Test cases for {@link RichAsyncFunction}
  */
 public class RichAsyncFunctionTest {
 
-	private RichAsyncFunction<String, String> initFunction() {
-		RichAsyncFunction<String, String> function = new RichAsyncFunction<String, String>() {
+	/**
+	 * Test the set of iteration runtime context methods in the context of a
+	 * {@link RichAsyncFunction}.
+	 */
+	@Test
+	public void testIterationRuntimeContext() throws Exception {
+		RichAsyncFunction<Integer, Integer> function = new RichAsyncFunction<Integer, Integer>() {
+			private static final long serialVersionUID = -2023923961609455894L;
+
 			@Override
-			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
-				getRuntimeContext().getState(mock(ValueStateDescriptor.class));
+			public void asyncInvoke(Integer input, AsyncCollector<Integer> collector) throws Exception {
+				// no op
 			}
 		};
 
-		return function;
+		int superstepNumber = 42;
+
+		IterationRuntimeContext mockedIterationRuntimeContext = mock(IterationRuntimeContext.class);
+		when(mockedIterationRuntimeContext.getSuperstepNumber()).thenReturn(superstepNumber);
+		function.setRuntimeContext(mockedIterationRuntimeContext);
+
+		IterationRuntimeContext iterationRuntimeContext = function.getIterationRuntimeContext();
+
+		assertEquals(superstepNumber, iterationRuntimeContext.getSuperstepNumber());
+
+		try {
+			iterationRuntimeContext.getIterationAggregator("foobar");
+			fail("Expected getIterationAggregator to fail with unsupported operation exception");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
+
+		try {
+			iterationRuntimeContext.getPreviousIterationAggregate("foobar");
+			fail("Expected getPreviousIterationAggregator to fail with unsupported operation exception");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 	}
 
+	/**
+	 * Test the set of runtime context methods in the context of a {@link RichAsyncFunction}.
+	 */
 	@Test
-	public void testIterationRuntimeContext() throws Exception {
-		// test runtime context is not set
-		RichAsyncFunction<String, String> function = new RichAsyncFunction<String, String>() {
+	public void testRuntimeContext() throws Exception {
+		RichAsyncFunction<Integer, Integer> function = new RichAsyncFunction<Integer, Integer>() {
+			private static final long serialVersionUID = 1707630162838967972L;
+
 			@Override
-			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
-				getIterationRuntimeContext().getIterationAggregator("test");
+			public void asyncInvoke(Integer input, AsyncCollector<Integer> collector) throws Exception {
+				// no op
 			}
 		};
 
+		final String taskName = "foobarTask";
+		final MetricGroup metricGroup = mock(MetricGroup.class);
+		final int numberOfParallelSubtasks = 42;
+		final int indexOfSubtask = 43;
+		final int attemptNumber = 1337;
+		final String taskNameWithSubtask = "barfoo";
+		final ExecutionConfig executionConfig = mock(ExecutionConfig.class);
+		final ClassLoader userCodeClassLoader = mock(ClassLoader.class);
+
+		RuntimeContext mockedRuntimeContext = mock(RuntimeContext.class);
+
+		when(mockedRuntimeContext.getTaskName()).thenReturn(taskName);
+		when(mockedRuntimeContext.getMetricGroup()).thenReturn(metricGroup);
+		when(mockedRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(numberOfParallelSubtasks);
+		when(mockedRuntimeContext.getIndexOfThisSubtask()).thenReturn(indexOfSubtask);
+		when(mockedRuntimeContext.getAttemptNumber()).thenReturn(attemptNumber);
+		when(mockedRuntimeContext.getTaskNameWithSubtasks()).thenReturn(taskNameWithSubtask);
+		when(mockedRuntimeContext.getExecutionConfig()).thenReturn(executionConfig);
+		when(mockedRuntimeContext.getUserCodeClassLoader()).thenReturn(userCodeClassLoader);
+
+		function.setRuntimeContext(mockedRuntimeContext);
+
+		RuntimeContext runtimeContext = function.getRuntimeContext();
+
+		assertEquals(taskName, runtimeContext.getTaskName());
+		assertEquals(metricGroup, runtimeContext.getMetricGroup());
+		assertEquals(numberOfParallelSubtasks, runtimeContext.getNumberOfParallelSubtasks());
+		assertEquals(indexOfSubtask, runtimeContext.getIndexOfThisSubtask());
+		assertEquals(attemptNumber, runtimeContext.getAttemptNumber());
+		assertEquals(taskNameWithSubtask, runtimeContext.getTaskNameWithSubtasks());
+		assertEquals(executionConfig, runtimeContext.getExecutionConfig());
+		assertEquals(userCodeClassLoader, runtimeContext.getUserCodeClassLoader());
+
 		try {
-			function.asyncInvoke("test", mock(AsyncCollector.class));
-		}
-		catch (Exception e) {
-			Assert.assertEquals("The runtime context has not been initialized.", e.getMessage());
+			runtimeContext.getDistributedCache();
+			fail("Expected getDistributedCached to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
 		}
 
-		// test get agg from iteration runtime context
-		function.setRuntimeContext(mock(IterationRuntimeContext.class));
-
 		try {
-			function.asyncInvoke("test", mock(AsyncCollector.class));
+			runtimeContext.getState(new ValueStateDescriptor<>("foobar", Integer.class, 42));
+			fail("Expected getState to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
 		}
-		catch (Exception e) {
-			Assert.assertEquals("Get iteration aggregator is not supported in rich async function", e.getMessage());
+
+		try {
+			runtimeContext.getListState(new ListStateDescriptor<>("foobar", Integer.class));
+			fail("Expected getListState to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
 		}
 
-		// get state from iteration runtime context
-		function = new RichAsyncFunction<String, String>() {
-			@Override
-			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
-				getIterationRuntimeContext().getState(mock(ValueStateDescriptor.class));
-			}
-		};
+		try {
+			runtimeContext.getReducingState(new ReducingStateDescriptor<>("foobar", new ReduceFunction<Integer>() {
+				private static final long serialVersionUID = 2136425961884441050L;
 
-		function.setRuntimeContext(mock(RuntimeContext.class));
+				@Override
+				public Integer reduce(Integer value1, Integer value2) throws Exception {
+					return value1;
+				}
+			}, Integer.class));
+			fail("Expected getReducingState to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 
 		try {
-			function.asyncInvoke("test", mock(AsyncCollector.class));
-		}
-		catch (Exception e) {
-			Assert.assertEquals("State is not supported in rich async function", e.getMessage());
-		}
+			runtimeContext.addAccumulator("foobar", new Accumulator<Integer, Integer>() {
+				private static final long serialVersionUID = -4673320336846482358L;
 
-		// test getting a counter from iteration runtime context
-		function = new RichAsyncFunction<String, String>() {
-			@Override
-			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
-				getIterationRuntimeContext().getIntCounter("test").add(6);
-			}
-		};
+				@Override
+				public void add(Integer value) {
+					// no op
+				}
 
-		IterationRuntimeContext context = mock(IterationRuntimeContext.class);
-		IntCounter counter = new IntCounter(0);
-		when(context.getIntCounter(anyString())).thenReturn(counter);
+				@Override
+				public Integer getLocalValue() {
+					return null;
+				}
 
-		function.setRuntimeContext(context);
+				@Override
+				public void resetLocal() {
 
-		function.asyncInvoke("test", mock(AsyncCollector.class));
+				}
 
-		Assert.assertTrue(6 == counter.getLocalValue());
-	}
+				@Override
+				public void merge(Accumulator<Integer, Integer> other) {
 
-	@Test
-	public void testRuntimeContext() throws Exception {
-		// test run time context is not set
-		RichAsyncFunction<String, String> function = new RichAsyncFunction<String, String>() {
-			@Override
-			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
-				getRuntimeContext().getState(mock(ValueStateDescriptor.class));
-			}
-		};
+				}
 
-		try {
-			function.asyncInvoke("test", mock(AsyncCollector.class));
-		}
-		catch (Exception e) {
-			Assert.assertEquals("The runtime context has not been initialized.", e.getMessage());
+				@Override
+				public Accumulator<Integer, Integer> clone() {
+					return null;
+				}
+			});
+			fail("Expected addAccumulator to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
 		}
 
-		// test get state
-		function = new RichAsyncFunction<String, String>() {
-			@Override
-			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
-				getRuntimeContext().getState(mock(ValueStateDescriptor.class));
-			}
-		};
+		try {
+			runtimeContext.getAccumulator("foobar");
+			fail("Expected getAccumulator to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 
-		function.setRuntimeContext(mock(RuntimeContext.class));
+		try {
+			runtimeContext.getAllAccumulators();
+			fail("Expected getAllAccumulators to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 
 		try {
-			function.asyncInvoke("test", mock(AsyncCollector.class));
+			runtimeContext.getIntCounter("foobar");
+			fail("Expected getIntCounter to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
 		}
-		catch (Exception e) {
-			Assert.assertEquals("State is not supported in rich async function", e.getMessage());
+
+		try {
+			runtimeContext.getLongCounter("foobar");
+			fail("Expected getLongCounter to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
 		}
 
-		// test getting a counter from runtime context
-		function = new RichAsyncFunction<String, String>() {
-			@Override
-			public void asyncInvoke(String input, AsyncCollector<String> collector) throws Exception {
-				getIterationRuntimeContext().getIntCounter("test").add(6);
-			}
-		};
+		try {
+			runtimeContext.getDoubleCounter("foobar");
+			fail("Expected getDoubleCounter to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 
-		IterationRuntimeContext context = mock(IterationRuntimeContext.class);
-		IntCounter counter = new IntCounter(0);
-		when(context.getIntCounter(anyString())).thenReturn(counter);
+		try {
+			runtimeContext.getHistogram("foobar");
+			fail("Expected getHistogram to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 
-		function.setRuntimeContext(context);
+		try {
+			runtimeContext.hasBroadcastVariable("foobar");
+			fail("Expected hasBroadcastVariable to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 
-		function.asyncInvoke("test", mock(AsyncCollector.class));
+		try {
+			runtimeContext.getBroadcastVariable("foobar");
+			fail("Expected getBroadcastVariable to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 
-		Assert.assertTrue(6 == counter.getLocalValue());
+		try {
+			runtimeContext.getBroadcastVariableWithInitializer("foobar", new BroadcastVariableInitializer<Object, Object>() {
+				@Override
+				public Object initializeBroadcastVariable(Iterable<Object> data) {
+					return null;
+				}
+			});
+			fail("Expected getBroadcastVariableWithInitializer to fail with unsupported operation exception.");
+		} catch (UnsupportedOperationException e) {
+			// expected
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ad603d59/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java
deleted file mode 100644
index d118d80..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBufferTest.java
+++ /dev/null
@@ -1,656 +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.async;
-
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.streaming.api.datastream.AsyncDataStream;
-import org.apache.flink.streaming.api.functions.async.buffer.StreamElementEntry;
-import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
-import org.apache.flink.streaming.api.functions.async.buffer.AsyncCollectorBuffer;
-import org.apache.flink.streaming.api.functions.async.AsyncFunction;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for {@link AsyncCollectorBuffer}. These test that:
- *
- * <ul>
- *     <li>Add a new item into the buffer</li>
- *     <li>Ordered mode processing</li>
- *     <li>Unordered mode processing</li>
- *     <li>Error handling</li>
- * </ul>
- */
-public class AsyncCollectorBufferTest {
-	private final static ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(10);
-
-	private final Random RANDOM = new Random();
-
-	private AsyncFunction<Integer, Integer> function;
-
-	private AsyncWaitOperator<Integer, Integer> operator;
-
-	private AsyncCollectorBuffer<Integer, Integer> buffer;
-
-	private Output<StreamRecord<Integer>> output;
-
-	private Object lock = new Object();
-
-	public AsyncCollectorBuffer<Integer, Integer> getBuffer(int bufferSize, AsyncDataStream.OutputMode mode) throws Exception {
-		function = new AsyncFunction<Integer, Integer>() {
-			@Override
-			public void asyncInvoke(Integer input, AsyncCollector<Integer> collector) throws Exception {
-
-			}
-		};
-
-		operator = new AsyncWaitOperator<>(function, bufferSize, mode);
-
-		StreamConfig cfg = new StreamConfig(new Configuration());
-		cfg.setTypeSerializerIn1(IntSerializer.INSTANCE);
-
-		StreamTask<?, ?> mockTask = mock(StreamTask.class);
-
-		when(mockTask.getCheckpointLock()).thenReturn(lock);
-
-		Environment env = new DummyEnvironment("DUMMY;-D", 1, 0);
-		when(mockTask.getEnvironment()).thenReturn(env);
-
-		output = new FakedOutput();
-
-		operator.setup(mockTask, cfg, output);
-
-		buffer = operator.getBuffer();
-
-		return buffer;
-	}
-
-	@Test
-	public void testAdd() throws Exception {
-		buffer = getBuffer(3, AsyncDataStream.OutputMode.ORDERED);
-
-		synchronized (lock) {
-			buffer.addWatermark(new Watermark(0l));
-			buffer.addLatencyMarker(new LatencyMarker(111L, 1, 1));
-		}
-
-		Assert.assertEquals(2, buffer.getQueue().size());
-
-		Iterator<StreamElementEntry<Integer>> iterator = buffer.getQueue().iterator();
-		Watermark watermark = iterator.next().getStreamElement().asWatermark();
-		Assert.assertEquals(0l, watermark.getTimestamp());
-
-		LatencyMarker latencyMarker = iterator.next().getStreamElement().asLatencyMarker();
-		Assert.assertEquals(111l, latencyMarker.getMarkedTime());
-
-		buffer.setExtraStreamElement(new Watermark(222l));
-
-		Iterator<StreamElement> elementIterator = buffer.getStreamElementsInBuffer();
-		Assert.assertEquals(0l, elementIterator.next().asWatermark().getTimestamp());
-		Assert.assertEquals(111l, elementIterator.next().asLatencyMarker().getMarkedTime());
-		Assert.assertEquals(222l, elementIterator.next().asWatermark().getTimestamp());
-		Assert.assertFalse(elementIterator.hasNext());
-	}
-
-	private void work(final boolean throwExcept) throws Exception {
-		final int ASYNC_COLLECTOR_NUM = 7;
-
-		Iterator<StreamElement> iterator = new Iterator<StreamElement>() {
-			private int idx = 0;
-
-			@Override
-			public boolean hasNext() {
-				return idx < ASYNC_COLLECTOR_NUM;
-			}
-
-			@Override
-			public StreamElement next() {
-				++idx;
-
-				if (idx == 4) {
-					return new Watermark(333l);
-				}
-				else if (idx == 7) {
-					return new LatencyMarker(111L, 0, 0);
-				}
-				else {
-					StreamRecord<Integer> ret = new StreamRecord<>(idx);
-					ret.setTimestamp(idx * idx);
-
-					return ret;
-				}
-			}
-
-			@Override
-			public void remove() {
-				// do nothing
-			}
-		};
-
-		while (iterator.hasNext()) {
-			final StreamElement record = iterator.next();
-
-			if (record.isRecord()) {
-				AsyncCollector tmp;
-
-				synchronized (lock) {
-					tmp = buffer.addStreamRecord(record.<Integer>asRecord());
-				}
-
-				final AsyncCollector collector = tmp;
-
-				EXECUTOR_SERVICE.submit(new Runnable() {
-					@Override
-					public void run() {
-						try {
-							Thread.sleep(RANDOM.nextInt(100));
-
-							if (throwExcept) {
-								collector.collect(new Exception("wahahahaha..."));
-							}
-							else {
-								collector.collect(Collections.singletonList(record.asRecord().getValue()));
-							}
-						} catch (InterruptedException e) {
-							// do nothing
-						}
-					}
-				});
-			}
-			else if (record.isWatermark()) {
-				synchronized (lock) {
-					buffer.addWatermark(record.asWatermark());
-				}
-			}
-			else {
-				synchronized (lock) {
-					buffer.addLatencyMarker(record.asLatencyMarker());
-				}
-			}
-		}
-	}
-
-	@Test
-	public void testOrderedBuffer() throws Exception {
-		buffer = getBuffer(3, AsyncDataStream.OutputMode.ORDERED);
-
-		buffer.startEmitterThread();
-
-		work(false);
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-		}
-
-		buffer.stopEmitterThread();
-
-		Assert.assertEquals("1,2,3,5,6,", ((FakedOutput)output).getValue());
-		Assert.assertEquals("1,4,9,333,25,36,111,", ((FakedOutput)output).getTimestamp());
-	}
-
-	@Test
-	public void testUnorderedBuffer() throws Exception {
-		buffer = getBuffer(3, AsyncDataStream.OutputMode.UNORDERED);
-
-		buffer.startEmitterThread();
-
-		work(false);
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-		}
-
-		buffer.stopEmitterThread();
-
-		Assert.assertEquals(333L, ((FakedOutput)output).getRawTimestamp().toArray()[3]);
-
-		List<Long> result = ((FakedOutput)output).getRawValue();
-		Collections.sort(result);
-		Assert.assertEquals("[1, 2, 3, 5, 6]", result.toString());
-
-		result = ((FakedOutput)output).getRawTimestamp();
-		Collections.sort(result);
-		Assert.assertEquals("[1, 4, 9, 25, 36, 111, 333]", result.toString());
-	}
-
-	@Test
-	public void testOrderedBufferWithManualTriggering() throws Exception {
-		// test AsyncCollectorBuffer with different combinations of StreamElements in the buffer.
-		// by triggering completion of each AsyncCollector one by one manually, we can verify
-		// the output one by one accurately.
-
-		FakedOutput fakedOutput;
-		AsyncCollector<Integer> collector1, collector2;
-
-		// 1. head element is a Watermark or LatencyMarker
-		buffer = getBuffer(3, AsyncDataStream.OutputMode.ORDERED);
-		fakedOutput = (FakedOutput)output;
-
-		fakedOutput.expect(1);
-
-		buffer.startEmitterThread();
-
-		synchronized (lock) {
-			buffer.addWatermark(new Watermark(1L));
-		}
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("", fakedOutput.getValue());
-		Assert.assertEquals("1,", fakedOutput.getTimestamp());
-
-
-		fakedOutput.expect(1);
-
-		synchronized (lock) {
-			buffer.addLatencyMarker(new LatencyMarker(2L, 0, 0));
-		}
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("", fakedOutput.getValue());
-		Assert.assertEquals("1,2,", fakedOutput.getTimestamp());
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-			buffer.stopEmitterThread();
-		}
-
-
-		// 2. buffer layout: WM -> SR1 -> LM -> SR2, where SR2 finishes first, then SR1.
-		buffer = getBuffer(5, AsyncDataStream.OutputMode.ORDERED);
-		fakedOutput = (FakedOutput)output;
-
-		synchronized (lock) {
-			buffer.addWatermark(new Watermark(1L));
-			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
-			buffer.addLatencyMarker(new LatencyMarker(3L, 0, 0));
-			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 4L));
-		}
-
-		fakedOutput.expect(1);
-
-		buffer.startEmitterThread();
-
-		fakedOutput.waitToFinish();
-
-		// in ORDERED mode, the result of completed SR2 will not be emitted right now.
-		collector2.collect(Collections.singletonList(222));
-
-		Thread.sleep(1000);
-
-		Assert.assertEquals("", fakedOutput.getValue());
-		Assert.assertEquals("1,", fakedOutput.getTimestamp());
-
-		fakedOutput.expect(3);
-
-		collector1.collect(Collections.singletonList(111));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("111,222,", fakedOutput.getValue());
-		Assert.assertEquals("1,2,3,4,", fakedOutput.getTimestamp());
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-			buffer.stopEmitterThread();
-		}
-
-		// 3. buffer layout: WM -> SR1 -> LM -> S2, where SR1 completes first, then SR2.
-		buffer = getBuffer(5, AsyncDataStream.OutputMode.ORDERED);
-		fakedOutput = (FakedOutput)output;
-
-		synchronized (lock) {
-			buffer.addWatermark(new Watermark(1L));
-			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
-			buffer.addLatencyMarker(new LatencyMarker(3L, 0, 0));
-			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 4L));
-		}
-
-		fakedOutput.expect(1);
-
-		buffer.startEmitterThread();
-
-		fakedOutput.waitToFinish();
-
-		fakedOutput.expect(2);
-
-		// in ORDERED mode, the result of completed SR1 will be emitted asap.
-		collector1.collect(Collections.singletonList(111));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("111,", fakedOutput.getValue());
-		Assert.assertEquals("1,2,3,", fakedOutput.getTimestamp());
-
-		fakedOutput.expect(1);
-
-		collector2.collect(Collections.singletonList(222));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("111,222,", fakedOutput.getValue());
-		Assert.assertEquals("1,2,3,4,", fakedOutput.getTimestamp());
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-			buffer.stopEmitterThread();
-		}
-
-		// 4. buffer layout: SR1 -> SR2 -> WM -> LM, where SR2 finishes first.
-		buffer = getBuffer(5, AsyncDataStream.OutputMode.ORDERED);
-		fakedOutput = (FakedOutput)output;
-
-		synchronized (lock) {
-			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 1L));
-			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 2L));
-			buffer.addWatermark(new Watermark(3L));
-			buffer.addLatencyMarker(new LatencyMarker(4L, 0, 0));
-		}
-
-		buffer.startEmitterThread();
-
-		// in ORDERED mode, the result of completed SR2 will not be emitted right now.
-		collector2.collect(Collections.singletonList(222));
-
-		Thread.sleep(1000);
-
-		Assert.assertEquals("", fakedOutput.getValue());
-		Assert.assertEquals("", fakedOutput.getTimestamp());
-
-		fakedOutput.expect(4);
-
-		collector1.collect(Collections.singletonList(111));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("111,222,", fakedOutput.getValue());
-		Assert.assertEquals("1,2,3,4,", fakedOutput.getTimestamp());
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-			buffer.stopEmitterThread();
-		}
-	}
-
-	@Test
-	public void testUnorderedWithManualTriggering() throws Exception {
-		// verify the output in UNORDERED mode by manual triggering.
-
-		FakedOutput fakedOutput;
-		AsyncCollector<Integer> collector1, collector2, collector3;
-
-		// 1. head element is a Watermark or LatencyMarker
-		buffer = getBuffer(5, AsyncDataStream.OutputMode.UNORDERED);
-		fakedOutput = (FakedOutput)output;
-
-		fakedOutput.expect(1);
-
-		buffer.startEmitterThread();
-
-		synchronized (lock) {
-			buffer.addWatermark(new Watermark(1L));
-		}
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("", fakedOutput.getValue());
-		Assert.assertEquals("1,", fakedOutput.getTimestamp());
-
-
-		fakedOutput.expect(1);
-
-		synchronized (lock) {
-			buffer.addLatencyMarker(new LatencyMarker(2L, 0, 0));
-		}
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("", fakedOutput.getValue());
-		Assert.assertEquals("1,2,", fakedOutput.getTimestamp());
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-			buffer.stopEmitterThread();
-		}
-
-
-		// 2. buffer layout: LM -> SR1 -> SR2 -> WM1 -> SR3 -> WM2, where the order of completion is SR3, SR2, SR1
-		buffer = getBuffer(6, AsyncDataStream.OutputMode.UNORDERED);
-		fakedOutput = (FakedOutput)output;
-
-		synchronized (lock) {
-			buffer.addLatencyMarker(new LatencyMarker(1L, 0, 0));
-			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
-			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 3L));
-			buffer.addWatermark(new Watermark(4L));
-			collector3 = buffer.addStreamRecord(new StreamRecord<>(333, 5L));
-			buffer.addWatermark(new Watermark(6L));
-		}
-
-		fakedOutput.expect(1);
-
-		buffer.startEmitterThread();
-
-		fakedOutput.waitToFinish();
-
-		// in UNORDERED mode, the result of completed SR3 will not be emitted right now.
-		collector3.collect(Collections.singletonList(333));
-
-		Thread.sleep(1000);
-
-		Assert.assertEquals("", fakedOutput.getValue());
-		Assert.assertEquals("1,", fakedOutput.getTimestamp());
-
-		fakedOutput.expect(1);
-
-		// SR2 will be emitted
-		collector2.collect(Collections.singletonList(222));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("222,", fakedOutput.getValue());
-		Assert.assertEquals("1,3,", fakedOutput.getTimestamp());
-
-		// SR1 will be emitted first, then WM, and then SR3 and WM2
-		fakedOutput.expect(4);
-		collector1.collect(Collections.singletonList(111));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("222,111,333,", fakedOutput.getValue());
-		Assert.assertEquals("1,3,2,4,5,6,", fakedOutput.getTimestamp());
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-			buffer.stopEmitterThread();
-		}
-
-		// 3. buffer layout: WM1 -> SR1 -> SR2 -> LM -> SR3 -> WM2, where the order of completion is SR2, SR1, SR3
-		buffer = getBuffer(6, AsyncDataStream.OutputMode.UNORDERED);
-		fakedOutput = (FakedOutput)output;
-
-		synchronized (lock) {
-			buffer.addWatermark(new Watermark(1L));
-			collector1 = buffer.addStreamRecord(new StreamRecord<>(111, 2L));
-			collector2 = buffer.addStreamRecord(new StreamRecord<>(222, 3L));
-			buffer.addLatencyMarker(new LatencyMarker(4L, 0, 0));
-			collector3 = buffer.addStreamRecord(new StreamRecord<>(333, 5L));
-			buffer.addWatermark(new Watermark(6L));
-		}
-
-		// the result of SR2 will be emitted following WM1
-		collector2.collect(Collections.singletonList(222));
-
-		fakedOutput.expect(2);
-
-		buffer.startEmitterThread();
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("222,", fakedOutput.getValue());
-		Assert.assertEquals("1,3,", fakedOutput.getTimestamp());
-
-		// SR1 and LM will be emitted
-		fakedOutput.expect(2);
-		collector1.collect(Collections.singletonList(111));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("222,111,", fakedOutput.getValue());
-		Assert.assertEquals("1,3,2,4,", fakedOutput.getTimestamp());
-
-		// SR3 and WM2 will be emitted
-		fakedOutput.expect(2);
-		collector3.collect(Collections.singletonList(333));
-
-		fakedOutput.waitToFinish();
-
-		Assert.assertEquals("222,111,333,", fakedOutput.getValue());
-		Assert.assertEquals("1,3,2,4,5,6,", fakedOutput.getTimestamp());
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-			buffer.stopEmitterThread();
-		}
-
-	}
-
-
-
-	@Test
-	public void testBufferWithException() throws Exception {
-		buffer = getBuffer(3, AsyncDataStream.OutputMode.UNORDERED);
-
-		buffer.startEmitterThread();
-
-		IOException expected = null;
-		try {
-			work(true);
-		}
-		catch (IOException e) {
-			expected = e;
-		}
-
-		Assert.assertNotNull(expected);
-		Assert.assertEquals(expected.getMessage(), "wahahahaha...");
-
-		synchronized (lock) {
-			buffer.waitEmpty();
-		}
-
-		buffer.stopEmitterThread();
-	}
-
-	public class FakedOutput implements Output<StreamRecord<Integer>> {
-		private List<Long> outputs;
-		private List<Long> timestamps;
-
-		private CountDownLatch latch;
-
-		public FakedOutput() {
-			this.outputs = new ArrayList<>();
-			this.timestamps = new ArrayList<>();
-		}
-
-		@Override
-		public void collect(StreamRecord<Integer> record) {
-			outputs.add(record.getValue().longValue());
-			if (record.hasTimestamp()) {
-				timestamps.add(record.getTimestamp());
-			}
-
-			if (latch != null) {
-				latch.countDown();
-			}
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			timestamps.add(mark.getTimestamp());
-
-			if (latch != null) {
-				latch.countDown();
-			}
-		}
-
-		@Override
-		public void emitLatencyMarker(LatencyMarker latencyMarker) {
-			timestamps.add(latencyMarker.getMarkedTime());
-
-			if (latch != null) {
-				latch.countDown();
-			}
-		}
-
-		@Override
-		public void close() {
-		}
-
-		public String getValue() {
-			StringBuilder sb = new StringBuilder();
-			for (Long i : outputs) {
-				sb.append(i).append(",");
-			}
-			return sb.toString();
-		}
-
-		public String getTimestamp() {
-			StringBuilder sb = new StringBuilder();
-			for (Long i : timestamps) {
-				sb.append(i).append(",");
-			}
-			return sb.toString();
-		}
-
-		public List<Long> getRawValue() {
-			return outputs;
-		}
-
-		public List<Long> getRawTimestamp() {
-			return timestamps;
-		}
-
-		public void expect(int count) {
-			latch = new CountDownLatch(count);
-		}
-
-		public void waitToFinish() throws InterruptedException {
-			latch.await();
-		}
-	}
-}