From 650c528fc4452e6b11294062699001a5b4a21ac9 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Tue, 17 Mar 2015 17:39:27 +0100 Subject: [PATCH 01/10] [FLINK-1706] Spilling BarrierBuffer added + basic tests --- .../runtime/io/network/buffer/Buffer.java | 4 + .../runtime/io/network/buffer/BufferTest.java | 1 + .../api/streamvertex/StreamingSuperstep.java | 8 ++ .../flink/streaming/io/BarrierBuffer.java | 54 +++++++---- .../flink/streaming/io/BufferSpiller.java | 85 +++++++++++++++++ .../flink/streaming/io/SpillReader.java | 82 ++++++++++++++++ .../streaming/io/SpillingBufferOrEvent.java | 94 +++++++++++++++++++ .../flink/streaming/io/BarrierBufferTest.java | 34 ++++--- .../io/SpillingBufferOrEventTest.java | 94 +++++++++++++++++++ 9 files changed, 424 insertions(+), 32 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillingBufferOrEvent.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/SpillingBufferOrEventTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java index 2ed82fa855286..264252120003f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java @@ -89,6 +89,10 @@ public ByteBuffer getNioBuffer() { return memorySegment.wrap(0, currentSize).duplicate(); } } + + public BufferRecycler getRecycler(){ + return recycler; + } public int getSize() { synchronized (recycleLock) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java index 2630608979590..f2f9c09501fa2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java @@ -50,4 +50,5 @@ public void testSetGetSize() { // OK => expected exception } } + } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingSuperstep.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingSuperstep.java index 557c6363156fe..d46ca793d24fa 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingSuperstep.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingSuperstep.java @@ -49,4 +49,12 @@ public void read(DataInputView in) throws IOException { public long getId() { return id; } + + public boolean equals(Object other) { + if (other == null || !(other instanceof StreamingSuperstep)) { + return false; + } else { + return ((StreamingSuperstep) other).id == this.id; + } + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java index 42d4919ed0025..ea422e5a753d8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java @@ -43,8 +43,8 @@ public class BarrierBuffer { private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class); - private Queue nonprocessed = new LinkedList(); - private Queue blockedNonprocessed = new LinkedList(); + private Queue nonprocessed = new LinkedList(); + private Queue blockedNonprocessed = new LinkedList(); private Set blockedChannels = new HashSet(); private int totalNumberOfInputChannels; @@ -56,10 +56,20 @@ public class BarrierBuffer { private InputGate inputGate; + private SpillReader spillReader; + private BufferSpiller bufferSpiller; + public BarrierBuffer(InputGate inputGate, AbstractReader reader) { this.inputGate = inputGate; totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); this.reader = reader; + try { + this.bufferSpiller = new BufferSpiller(); + this.spillReader = new SpillReader(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } /** @@ -76,29 +86,24 @@ protected void startSuperstep(StreamingSuperstep superstep) { } } - /** - * Buffers a bufferOrEvent received from a blocked channel - * - * @param bufferOrEvent - * bufferOrEvent to buffer - */ - protected void store(BufferOrEvent bufferOrEvent) { - nonprocessed.add(bufferOrEvent); - } - /** * Get then next non-blocked non-processed BufferOrEvent. Returns null if * not available. + * + * @throws IOException */ - protected BufferOrEvent getNonProcessed() { - BufferOrEvent nextNonprocessed; + protected BufferOrEvent getNonProcessed() throws IOException { + SpillingBufferOrEvent nextNonprocessed; + while ((nextNonprocessed = nonprocessed.poll()) != null) { - if (isBlocked(nextNonprocessed.getChannelIndex())) { - blockedNonprocessed.add(nextNonprocessed); + BufferOrEvent boe = nextNonprocessed.getBufferOrEvent(); + if (isBlocked(boe.getChannelIndex())) { + blockedNonprocessed.add(new SpillingBufferOrEvent(boe, bufferSpiller, spillReader)); } else { - return nextNonprocessed; + return boe; } } + return null; } @@ -137,7 +142,8 @@ public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedExceptio bufferOrEvent = inputGate.getNextBufferOrEvent(); if (isBlocked(bufferOrEvent.getChannelIndex())) { // If channel blocked we just store it - blockedNonprocessed.add(bufferOrEvent); + blockedNonprocessed.add(new SpillingBufferOrEvent(bufferOrEvent, bufferSpiller, + spillReader)); } else { return bufferOrEvent; } @@ -168,6 +174,8 @@ protected void blockChannel(int channelIndex) { /** * Releases the blocks on all channels. + * + * @throws IOException */ protected void releaseBlocks() { if (!nonprocessed.isEmpty()) { @@ -175,7 +183,15 @@ protected void releaseBlocks() { throw new RuntimeException("Error in barrier buffer logic"); } nonprocessed = blockedNonprocessed; - blockedNonprocessed = new LinkedList(); + blockedNonprocessed = new LinkedList(); + + try { + spillReader.setSpillFile(bufferSpiller.getSpillFile()); + bufferSpiller.resetSpillFile(); + } catch (IOException e) { + throw new RuntimeException(e); + } + blockedChannels.clear(); superstepStarted = false; if (LOG.isDebugEnabled()) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.java new file mode 100644 index 0000000000000..e824430ab3cd5 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.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.io; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; +import java.util.Random; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.util.StringUtils; + +public class BufferSpiller { + + protected static Random rnd = new Random(); + + private File spillFile; + protected FileChannel spillingChannel; + private String tempDir; + + public BufferSpiller() throws IOException { + String tempDirString = GlobalConfiguration.getString( + ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH); + String[] tempDirs = tempDirString.split(",|" + File.pathSeparator); + + tempDir = tempDirs[rnd.nextInt(tempDirs.length)]; + + createSpillingChannel(); + } + + /** + * Dumps the contents of the buffer to disk and recycles the buffer. + */ + public void spill(Buffer buffer) throws IOException { + spillingChannel.write(buffer.getNioBuffer()); + buffer.recycle(); + } + + @SuppressWarnings("resource") + private void createSpillingChannel() throws IOException { + this.spillFile = new File(tempDir, randomString(rnd) + ".buffer"); + this.spillingChannel = new RandomAccessFile(spillFile, "rw").getChannel(); + } + + private static String randomString(Random random) { + final byte[] bytes = new byte[20]; + random.nextBytes(bytes); + return StringUtils.byteToHexString(bytes); + } + + public void close() throws IOException { + if (spillingChannel != null && spillingChannel.isOpen()) { + spillingChannel.close(); + } + } + + public void resetSpillFile() throws IOException { + close(); + createSpillingChannel(); + } + + public File getSpillFile() { + return spillFile; + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.java new file mode 100644 index 0000000000000..3526bea349b4c --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.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.io; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; + +public class SpillReader { + + private FileChannel spillingChannel; + private File spillFile; + + /** + * Reads the next buffer from the spilled file. If a buffer pool was given, + * uses the buffer pool to request a new buffer to read into. + * + */ + public Buffer readNextBuffer(int bufferSize, BufferPool bufferPool) throws IOException { + + Buffer buffer = null; + + // If available tries to request a new buffer from the pool + if (bufferPool != null) { + buffer = bufferPool.requestBuffer(); + } + + // If no bufferpool provided or the pool was empty create a new buffer + if (buffer == null) { + buffer = new Buffer(new MemorySegment(new byte[bufferSize]), new BufferRecycler() { + + @Override + public void recycle(MemorySegment memorySegment) { + memorySegment.free(); + } + }); + } + + spillingChannel.read(buffer.getMemorySegment().wrap(0, bufferSize)); + + return buffer; + } + + @SuppressWarnings("resource") + public void setSpillFile(File nextSpillFile) throws IOException { + // We can close and delete the file now + close(); + if (spillFile != null) { + spillFile.delete(); + } + this.spillFile = nextSpillFile; + this.spillingChannel = new RandomAccessFile(spillFile, "rw").getChannel(); + } + + public void close() throws IOException { + if (this.spillingChannel != null && this.spillingChannel.isOpen()) { + this.spillingChannel.close(); + } + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillingBufferOrEvent.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillingBufferOrEvent.java new file mode 100644 index 0000000000000..40713e2b5caa1 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillingBufferOrEvent.java @@ -0,0 +1,94 @@ +/* + * 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.io; + +import java.io.IOException; + +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; + +public class SpillingBufferOrEvent { + + private BufferOrEvent boe; + private boolean isSpilled = false; + + private SpillReader spillReader; + + private BufferPool bufferPool; + + private int channelIndex; + private int bufferSize; + + public SpillingBufferOrEvent(BufferOrEvent boe, BufferSpiller spiller, SpillReader reader) + throws IOException { + + this.boe = boe; + this.spillReader = reader; + + if (shouldSpill()) { + spiller.spill(boe.getBuffer()); + isSpilled = true; + boe = null; + } + } + + /** + * If the buffer wasn't spilled simply returns the instance from the field, + * otherwise reads it from the spill reader + */ + public BufferOrEvent getBufferOrEvent() throws IOException { + if (isSpilled) { + return new BufferOrEvent(spillReader.readNextBuffer(bufferSize, bufferPool), + channelIndex); + } else { + return boe; + } + } + + /** + * Checks whether a given buffer should be spilled to disk. Currently it + * checks whether the buffer pool from which the buffer was supplied is + * empty and only spills if it is. This avoids out of memory exceptions and + * also blocks at the input gate. + */ + private boolean shouldSpill() throws IOException { + if (boe.isBuffer()) { + Buffer buffer = boe.getBuffer(); + this.bufferSize = buffer.getSize(); + BufferRecycler recycler = buffer.getRecycler(); + + if (recycler instanceof BufferPool) { + bufferPool = (BufferPool) recycler; + Buffer nextBuffer = bufferPool.requestBuffer(); + if (nextBuffer == null) { + return true; + } else { + nextBuffer.recycle(); + } + } + } + + return false; + } + + public boolean isSpilled() { + return isSpilled; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java index 1b4cc36ac891d..2b8a218f73688 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java @@ -110,28 +110,36 @@ public void testMultiChannelBarrier() throws IOException, InterruptedException { BarrierBuffer bb = new BarrierBuffer(mockIG1, mockAR1); BufferOrEvent nextBoe; - assertEquals(input.get(0), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(1), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(2), nextBoe = bb.getNextNonBlocked()); + check(input.get(0), nextBoe = bb.getNextNonBlocked()); + check(input.get(1), nextBoe = bb.getNextNonBlocked()); + check(input.get(2), nextBoe = bb.getNextNonBlocked()); bb.processSuperstep(nextBoe); - assertEquals(input.get(7), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(8), nextBoe = bb.getNextNonBlocked()); + check(input.get(7), nextBoe = bb.getNextNonBlocked()); + check(input.get(8), nextBoe = bb.getNextNonBlocked()); bb.processSuperstep(nextBoe); - assertEquals(input.get(3), nextBoe = bb.getNextNonBlocked()); + check(input.get(3), nextBoe = bb.getNextNonBlocked()); bb.processSuperstep(nextBoe); - assertEquals(input.get(10), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(11), nextBoe = bb.getNextNonBlocked()); + check(input.get(10), nextBoe = bb.getNextNonBlocked()); + check(input.get(11), nextBoe = bb.getNextNonBlocked()); bb.processSuperstep(nextBoe); - assertEquals(input.get(4), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(5), nextBoe = bb.getNextNonBlocked()); + check(input.get(4), nextBoe = bb.getNextNonBlocked()); + check(input.get(5), nextBoe = bb.getNextNonBlocked()); bb.processSuperstep(nextBoe); - assertEquals(input.get(12), nextBoe = bb.getNextNonBlocked()); + check(input.get(12), nextBoe = bb.getNextNonBlocked()); bb.processSuperstep(nextBoe); - assertEquals(input.get(6), nextBoe = bb.getNextNonBlocked()); - assertEquals(input.get(9), nextBoe = bb.getNextNonBlocked()); + check(input.get(6), nextBoe = bb.getNextNonBlocked()); + check(input.get(9), nextBoe = bb.getNextNonBlocked()); } + private static void check(BufferOrEvent expected, BufferOrEvent actual) { + assertEquals(expected.isBuffer(), actual.isBuffer()); + assertEquals(expected.getChannelIndex(), actual.getChannelIndex()); + if (expected.isEvent()) { + assertEquals(expected.getEvent(), actual.getEvent()); + } + } + protected static class MockInputGate implements InputGate { private int numChannels; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/SpillingBufferOrEventTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/SpillingBufferOrEventTest.java new file mode 100644 index 0000000000000..2f28f90639361 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/SpillingBufferOrEventTest.java @@ -0,0 +1,94 @@ +/* + * 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.io; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; + +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.junit.Test; + +public class SpillingBufferOrEventTest { + + @Test + public void testSpilling() throws IOException, InterruptedException { + BufferSpiller bsp = new BufferSpiller(); + SpillReader spr = new SpillReader(); + + BufferPool pool1 = new NetworkBufferPool(10, 256).createBufferPool(2, true); + BufferPool pool2 = new NetworkBufferPool(10, 256).createBufferPool(2, true); + + Buffer b1 = pool1.requestBuffer(); + b1.getMemorySegment().putInt(0, 10000); + BufferOrEvent boe1 = new BufferOrEvent(b1, 0); + SpillingBufferOrEvent sboe1 = new SpillingBufferOrEvent(boe1, bsp, spr); + + assertFalse(sboe1.isSpilled()); + + Buffer b2 = pool2.requestBuffer(); + b2.getMemorySegment().putInt(0, 10000); + BufferOrEvent boe2 = new BufferOrEvent(b2, 0); + SpillingBufferOrEvent sboe2 = new SpillingBufferOrEvent(boe2, bsp, spr); + + assertFalse(sboe2.isSpilled()); + + Buffer b3 = pool1.requestBuffer(); + b3.getMemorySegment().putInt(0, 50000); + BufferOrEvent boe3 = new BufferOrEvent(b3, 0); + SpillingBufferOrEvent sboe3 = new SpillingBufferOrEvent(boe3, bsp, spr); + + assertTrue(sboe3.isSpilled()); + + Buffer b4 = pool2.requestBuffer(); + b4.getMemorySegment().putInt(0, 60000); + BufferOrEvent boe4 = new BufferOrEvent(b4, 0); + SpillingBufferOrEvent sboe4 = new SpillingBufferOrEvent(boe4, bsp, spr); + + assertTrue(sboe4.isSpilled()); + + bsp.close(); + + spr.setSpillFile(bsp.getSpillFile()); + + Buffer b1ret = sboe1.getBufferOrEvent().getBuffer(); + assertEquals(10000, b1ret.getMemorySegment().getInt(0)); + b1ret.recycle(); + + Buffer b2ret = sboe2.getBufferOrEvent().getBuffer(); + assertEquals(10000, b2ret.getMemorySegment().getInt(0)); + b2ret.recycle(); + + Buffer b3ret = sboe3.getBufferOrEvent().getBuffer(); + assertEquals(50000, b3ret.getMemorySegment().getInt(0)); + b3ret.recycle(); + + Buffer b4ret = sboe4.getBufferOrEvent().getBuffer(); + assertEquals(60000, b4ret.getMemorySegment().getInt(0)); + b4ret.recycle(); + + spr.close(); + bsp.getSpillFile().delete(); + + } +} From f625b8ddadef2268e692905677e69a0adb1713ce Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Wed, 18 Mar 2015 19:22:16 +0100 Subject: [PATCH 02/10] [FLINK-1706] Resource cleanup added streaming readers with BarrierBuffers --- .../api/streamvertex/CoStreamVertex.java | 4 +- .../api/streamvertex/InputHandler.java | 5 ++- .../api/streamvertex/StreamVertex.java | 2 +- .../flink/streaming/io/BarrierBuffer.java | 15 +++++++ .../flink/streaming/io/BufferSpiller.java | 10 ++++- .../flink/streaming/io/CoRecordReader.java | 14 ++++-- .../flink/streaming/io/SpillReader.java | 43 +++++++++++-------- .../io/StreamingAbstractRecordReader.java | 8 +++- .../flink/streaming/io/StreamingReader.java | 27 ++++++++++++ .../flink/streaming/io/BarrierBufferTest.java | 3 ++ 10 files changed, 104 insertions(+), 27 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingReader.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java index 69576528be00b..d794a35fd459c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.io.IndexedReaderIterator; import org.apache.flink.util.MutableObjectIterator; +import java.io.IOException; import java.util.ArrayList; public class CoStreamVertex extends StreamVertex { @@ -63,9 +64,10 @@ public void setInputsOutputs() { } @Override - public void clearBuffers() { + public void clearBuffers() throws IOException { outputHandler.clearWriters(); coReader.clearBuffers(); + coReader.cleanup(); } protected void setConfigInputs() throws StreamVertexException { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java index d766705f7c5c1..726df53e26ffd 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java @@ -17,6 +17,8 @@ package org.apache.flink.streaming.api.streamvertex; +import java.io.IOException; + import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.io.network.api.reader.MutableReader; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; @@ -86,9 +88,10 @@ public IndexedReaderIterator> getInputIter() { return inputIter; } - public void clearReaders() { + public void clearReaders() throws IOException { if (inputs != null) { inputs.clearBuffers(); } + inputs.cleanup(); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java index 926aac220c0cc..30b69a77cf153 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java @@ -191,7 +191,7 @@ public void invoke() throws Exception { } - protected void clearBuffers() { + protected void clearBuffers() throws IOException { if (outputHandler != null) { outputHandler.clearWriters(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java index ea422e5a753d8..f349ac501ecd9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BarrierBuffer.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.io; +import java.io.File; import java.io.IOException; import java.util.HashSet; import java.util.LinkedList; @@ -225,4 +226,18 @@ public void processSuperstep(BufferOrEvent bufferOrEvent) { blockChannel(bufferOrEvent.getChannelIndex()); } + public void cleanup() throws IOException { + bufferSpiller.close(); + File spillfile1 = bufferSpiller.getSpillFile(); + if (spillfile1 != null) { + spillfile1.delete(); + } + + spillReader.close(); + File spillfile2 = spillReader.getSpillFile(); + if (spillfile2 != null) { + spillfile2.delete(); + } + } + } \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.java index e824430ab3cd5..b028ea733fbe6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/BufferSpiller.java @@ -51,8 +51,14 @@ public BufferSpiller() throws IOException { * Dumps the contents of the buffer to disk and recycles the buffer. */ public void spill(Buffer buffer) throws IOException { - spillingChannel.write(buffer.getNioBuffer()); - buffer.recycle(); + try { + spillingChannel.write(buffer.getNioBuffer()); + buffer.recycle(); + } catch (IOException e) { + close(); + throw new IOException(e); + } + } @SuppressWarnings("resource") diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoRecordReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoRecordReader.java index c32db4ec41418..25cb25d5d89a1 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoRecordReader.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoRecordReader.java @@ -39,7 +39,7 @@ */ @SuppressWarnings("rawtypes") public class CoRecordReader extends - AbstractReader implements EventListener { + AbstractReader implements EventListener, StreamingReader { private final InputGate bufferReader1; @@ -232,8 +232,8 @@ protected int getNextReaderIndexBlocking() throws InterruptedException { public void onEvent(InputGate bufferReader) { addToAvailable(bufferReader); } - - protected void addToAvailable(InputGate bufferReader){ + + protected void addToAvailable(InputGate bufferReader) { if (bufferReader == bufferReader1) { availableRecordReaders.add(1); } else if (bufferReader == bufferReader2) { @@ -278,4 +278,12 @@ protected void actOnAllBlocked() { } + public void cleanup() throws IOException { + try { + barrierBuffer1.cleanup(); + } finally { + barrierBuffer2.cleanup(); + } + + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.java index 3526bea349b4c..3cb83d4551016 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/SpillReader.java @@ -38,28 +38,33 @@ public class SpillReader { * */ public Buffer readNextBuffer(int bufferSize, BufferPool bufferPool) throws IOException { + try { + Buffer buffer = null; - Buffer buffer = null; + // If available tries to request a new buffer from the pool + if (bufferPool != null) { + buffer = bufferPool.requestBuffer(); + } - // If available tries to request a new buffer from the pool - if (bufferPool != null) { - buffer = bufferPool.requestBuffer(); - } - - // If no bufferpool provided or the pool was empty create a new buffer - if (buffer == null) { - buffer = new Buffer(new MemorySegment(new byte[bufferSize]), new BufferRecycler() { + // If no bufferpool provided or the pool was empty create a new + // buffer + if (buffer == null) { + buffer = new Buffer(new MemorySegment(new byte[bufferSize]), new BufferRecycler() { - @Override - public void recycle(MemorySegment memorySegment) { - memorySegment.free(); - } - }); - } + @Override + public void recycle(MemorySegment memorySegment) { + memorySegment.free(); + } + }); + } - spillingChannel.read(buffer.getMemorySegment().wrap(0, bufferSize)); + spillingChannel.read(buffer.getMemorySegment().wrap(0, bufferSize)); - return buffer; + return buffer; + } catch (Exception e) { + close(); + throw new IOException(e); + } } @SuppressWarnings("resource") @@ -73,6 +78,10 @@ public void setSpillFile(File nextSpillFile) throws IOException { this.spillingChannel = new RandomAccessFile(spillFile, "rw").getChannel(); } + public File getSpillFile() { + return spillFile; + } + public void close() throws IOException { if (this.spillingChannel != null && this.spillingChannel.isOpen()) { this.spillingChannel.close(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java index 82960101ecd84..c5ffa62ebd314 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingAbstractRecordReader.java @@ -43,8 +43,8 @@ * @param * The type of the record that can be read with this record reader. */ -public abstract class StreamingAbstractRecordReader extends AbstractReader implements - ReaderBase { +public abstract class StreamingAbstractRecordReader extends + AbstractReader implements ReaderBase, StreamingReader { @SuppressWarnings("unused") private static final Logger LOG = LoggerFactory.getLogger(StreamingAbstractRecordReader.class); @@ -122,4 +122,8 @@ public void clearBuffers() { } } } + + public void cleanup() throws IOException { + barrierBuffer.cleanup(); + } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingReader.java new file mode 100644 index 0000000000000..74b986a3f6a3e --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamingReader.java @@ -0,0 +1,27 @@ +/* + * 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.io; + +import java.io.IOException; + +public interface StreamingReader { + + public void cleanup() throws IOException; + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java index 2b8a218f73688..d8749023f44f3 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferTest.java @@ -54,6 +54,7 @@ public void testWithoutBarriers() throws IOException, InterruptedException { assertEquals(input.get(1), bb.getNextNonBlocked()); assertEquals(input.get(2), bb.getNextNonBlocked()); + bb.cleanup(); } @Test @@ -84,6 +85,7 @@ public void testOneChannelBarrier() throws IOException, InterruptedException { bb.processSuperstep(nextBoe); assertEquals(input.get(6), nextBoe = bb.getNextNonBlocked()); + bb.cleanup(); } @Test @@ -130,6 +132,7 @@ public void testMultiChannelBarrier() throws IOException, InterruptedException { check(input.get(6), nextBoe = bb.getNextNonBlocked()); check(input.get(9), nextBoe = bb.getNextNonBlocked()); + bb.cleanup(); } private static void check(BufferOrEvent expected, BufferOrEvent actual) { From ceea93d41b6e8ae66cd96d03f2f27e0027f8ac4c Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Wed, 18 Mar 2015 23:41:08 +0100 Subject: [PATCH 03/10] [FLINK-1706] IOTest added for BarrierBuffers Closes #493 --- .../api/streamvertex/InputHandler.java | 2 +- .../api/streamvertex/StreamVertex.java | 5 +- .../streaming/io/BarrierBufferIOTest.java | 159 ++++++++++++++++++ 3 files changed, 162 insertions(+), 4 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferIOTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java index 726df53e26ffd..48a00d8708c8a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java @@ -91,7 +91,7 @@ public IndexedReaderIterator> getInputIter() { public void clearReaders() throws IOException { if (inputs != null) { inputs.clearBuffers(); + inputs.cleanup(); } - inputs.cleanup(); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java index 30b69a77cf153..dd8a463e9f626 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java @@ -114,9 +114,8 @@ public void confirmBarrier(long barrierID) throws IOException { if (configuration.getStateMonitoring() && !states.isEmpty()) { getEnvironment().getJobManager().tell( new StateBarrierAck(getEnvironment().getJobID(), getEnvironment() - .getJobVertexId(), context.getIndexOfThisSubtask(), barrierID, - new LocalStateHandle(states)), - ActorRef.noSender()); + .getJobVertexId(), context.getIndexOfThisSubtask(), barrierID, + new LocalStateHandle(states)), ActorRef.noSender()); } else { getEnvironment().getJobManager().tell( new BarrierAck(getEnvironment().getJobID(), getEnvironment().getJobVertexId(), diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferIOTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferIOTest.java new file mode 100644 index 0000000000000..24106c1c23778 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/io/BarrierBufferIOTest.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.io; + +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.Random; + +import org.apache.flink.runtime.event.task.TaskEvent; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.util.event.EventListener; +import org.junit.Test; + +public class BarrierBufferIOTest { + + @Test + public void IOTest() throws IOException, InterruptedException { + + BufferPool pool1 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); + BufferPool pool2 = new NetworkBufferPool(100, 1024).createBufferPool(100, true); + + MockInputGate myIG = new MockInputGate(new BufferPool[] { pool1, pool2 }, + new BarrierGenerator[] { new CountBarrier(100000), new RandomBarrier(100000) }); + // new BarrierSimulator[] { new CountBarrier(1000), new + // CountBarrier(1000) }); + + BarrierBuffer barrierBuffer = new BarrierBuffer(myIG, + new BarrierBufferTest.MockReader(myIG)); + + try { + // long time = System.currentTimeMillis(); + for (int i = 0; i < 2000000; i++) { + BufferOrEvent boe = barrierBuffer.getNextNonBlocked(); + if (boe.isBuffer()) { + boe.getBuffer().recycle(); + } else { + barrierBuffer.processSuperstep(boe); + } + } + // System.out.println("Ran for " + (System.currentTimeMillis() - + // time)); + } catch (Exception e) { + fail(); + } finally { + barrierBuffer.cleanup(); + } + } + + private static class RandomBarrier implements BarrierGenerator { + private static Random rnd = new Random(); + + double threshold; + + public RandomBarrier(double expectedEvery) { + threshold = 1 / expectedEvery; + } + + @Override + public boolean isNextBarrier() { + return rnd.nextDouble() < threshold; + } + } + + private static class CountBarrier implements BarrierGenerator { + + long every; + long c = 0; + + public CountBarrier(long every) { + this.every = every; + } + + @Override + public boolean isNextBarrier() { + return c++ % every == 0; + } + } + + protected static class MockInputGate implements InputGate { + + private int numChannels; + private BufferPool[] bufferPools; + private int[] currentSupersteps; + BarrierGenerator[] barrierGens; + int currentChannel = 0; + long c = 0; + + public MockInputGate(BufferPool[] bufferPools, BarrierGenerator[] barrierGens) { + this.numChannels = bufferPools.length; + this.currentSupersteps = new int[numChannels]; + this.bufferPools = bufferPools; + this.barrierGens = barrierGens; + } + + @Override + public int getNumberOfInputChannels() { + return numChannels; + } + + @Override + public boolean isFinished() { + return false; + } + + @Override + public void requestPartitions() throws IOException, InterruptedException { + } + + @Override + public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException { + currentChannel = (currentChannel + 1) % numChannels; + + if (barrierGens[currentChannel].isNextBarrier()) { + return BarrierBufferTest.createSuperstep(++currentSupersteps[currentChannel], + currentChannel); + } else { + Buffer buffer = bufferPools[currentChannel].requestBuffer(); + buffer.getMemorySegment().putLong(0, c++); + + return new BufferOrEvent(buffer, currentChannel); + } + + } + + @Override + public void sendTaskEvent(TaskEvent event) throws IOException { + } + + @Override + public void registerListener(EventListener listener) { + } + + } + + protected interface BarrierGenerator { + public boolean isNextBarrier(); + } + +} From 43d0c71d59e677dd35f1c4b3e5b30f27589aaab6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Fri, 13 Mar 2015 09:59:09 +0100 Subject: [PATCH 04/10] [streaming] Added thread-safe list to tests --- .../flink/streaming/api/CoStreamTest.java | 97 ++++++++++++++++ .../streaming/api/OutputSplitterTest.java | 106 ++++-------------- .../streaming/api/WindowCrossJoinTest.java | 67 +++++------ .../api/collector/DirectedOutputTest.java | 39 ++++--- .../streaming/util/TestListResultSink.java | 78 +++++++++++++ .../flink/streaming/util/TestListWrapper.java | 60 ++++++++++ 6 files changed, 309 insertions(+), 138 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java new file mode 100644 index 0000000000000..20cd18914141d --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.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; + +import static org.junit.Assert.assertEquals; + +import java.util.ArrayList; +import java.util.Arrays; + +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.function.co.CoMapFunction; +import org.apache.flink.streaming.util.TestListResultSink; +import org.apache.flink.streaming.util.TestStreamEnvironment; + +public class CoStreamTest { + + private static final long MEMORY_SIZE = 32; + + private static ArrayList expected; + + public static void main(String[] args) throws InterruptedException { + for (int i = 0; i < 200; i++) { + test(); + } + } + + // @Test + public static void test() { + expected = new ArrayList(); + + StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE); + + TestListResultSink resultSink = new TestListResultSink(); + + DataStream src = env.fromElements(1, 3, 5); + DataStream src2 = env.fromElements(1, 3, 5); + + DataStream grouped = src.groupBy(new KeySelector() { + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + }); + + DataStream grouped2 = src2.groupBy(new KeySelector() { + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + }); + + DataStream connected = grouped.connect(grouped2).map(new CoMapFunction() { + @Override + public String map1(Integer value) { + return value.toString(); + } + + @Override + public String map2(Integer value) { + return value.toString(); + } + }); + + connected.addSink(resultSink); + + connected.print(); + + try { + env.execute(); + } catch (Exception e) { + e.printStackTrace(); + } + + expected = new ArrayList(); + expected.addAll(Arrays.asList("1", "1", "3", "3", "5", "5")); + + System.out.println(resultSink.getResult()); + assertEquals(expected, expected); + } +} \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java index a214fbf64bb16..cf6bb3cacfe35 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java @@ -19,39 +19,34 @@ import static org.junit.Assert.assertEquals; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + import org.apache.flink.streaming.api.collector.OutputSelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.function.sink.SinkFunction; +import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - public class OutputSplitterTest { private static final long MEMORYSIZE = 32; - private static ArrayList splitterResult1 = new ArrayList(); - private static ArrayList splitterResult2 = new ArrayList(); - - private static ArrayList expectedSplitterResult = new ArrayList(); @SuppressWarnings("unchecked") @Test public void testOnMergedDataStream() throws Exception { - splitterResult1.clear(); - splitterResult2.clear(); + TestListResultSink splitterResultSink1 = new TestListResultSink(); + TestListResultSink splitterResultSink2 = new TestListResultSink(); StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE); env.setBufferTimeout(1); - DataStream d1 = env.fromElements(0,2,4,6,8); - DataStream d2 = env.fromElements(1,3,5,7,9); + DataStream d1 = env.fromElements(0, 2, 4, 6, 8); + DataStream d2 = env.fromElements(1, 3, 5, 7, 9); d1 = d1.merge(d2); @@ -68,19 +63,7 @@ public Iterable select(Integer value) { } return s; } - }).select(">").addSink(new SinkFunction() { - - private static final long serialVersionUID = 5827187510526388104L; - - @Override - public void invoke(Integer value) { - splitterResult1.add(value); - } - - @Override - public void cancel() { - } - }); + }).select(">").addSink(splitterResultSink1); d1.split(new OutputSelector() { private static final long serialVersionUID = -6822487543355994807L; @@ -95,41 +78,27 @@ public Iterable select(Integer value) { } return s; } - }).select("yes").addSink(new SinkFunction() { - private static final long serialVersionUID = -2674335071267854599L; - - @Override - public void invoke(Integer value) { - splitterResult2.add(value); - } - - @Override - public void cancel() { - } - }); + }).select("yes").addSink(splitterResultSink2); env.execute(); - Collections.sort(splitterResult1); - Collections.sort(splitterResult2); - expectedSplitterResult.clear(); - expectedSplitterResult.addAll(Arrays.asList(5,6,7,8,9)); - assertEquals(expectedSplitterResult, splitterResult1); + expectedSplitterResult.addAll(Arrays.asList(5, 6, 7, 8, 9)); + assertEquals(expectedSplitterResult, splitterResultSink1.getSortedResult()); expectedSplitterResult.clear(); - expectedSplitterResult.addAll(Arrays.asList(0,3,6,9)); - assertEquals(expectedSplitterResult, splitterResult2); + expectedSplitterResult.addAll(Arrays.asList(0, 3, 6, 9)); + assertEquals(expectedSplitterResult, splitterResultSink2.getSortedResult()); } @Test public void testOnSingleDataStream() throws Exception { - splitterResult1.clear(); - splitterResult2.clear(); + TestListResultSink splitterResultSink1 = new TestListResultSink(); + TestListResultSink splitterResultSink2 = new TestListResultSink(); StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE); env.setBufferTimeout(1); - DataStream ds = env.fromElements(0,1,2,3,4,5,6,7,8,9); + DataStream ds = env.fromElements(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); ds.split(new OutputSelector() { private static final long serialVersionUID = 2524335410904414121L; @@ -144,19 +113,7 @@ public Iterable select(Integer value) { } return s; } - }).select("even").addSink(new SinkFunction() { - - private static final long serialVersionUID = -2995092337537209535L; - - @Override - public void invoke(Integer value) { - splitterResult1.add(value); - } - - @Override - public void cancel() { - } - }); + }).select("even").addSink(splitterResultSink1); ds.split(new OutputSelector() { @@ -172,30 +129,15 @@ public Iterable select(Integer value) { } return s; } - }).select("yes").addSink(new SinkFunction() { - - private static final long serialVersionUID = -1749077049727705424L; - - @Override - public void invoke(Integer value) { - splitterResult2.add(value); - } - - @Override - public void cancel() { - } - }); + }).select("yes").addSink(splitterResultSink2); env.execute(); - Collections.sort(splitterResult1); - Collections.sort(splitterResult2); - expectedSplitterResult.clear(); - expectedSplitterResult.addAll(Arrays.asList(0,2,4,6,8)); - assertEquals(expectedSplitterResult, splitterResult1); + expectedSplitterResult.addAll(Arrays.asList(0, 2, 4, 6, 8)); + assertEquals(expectedSplitterResult, splitterResultSink1.getSortedResult()); expectedSplitterResult.clear(); - expectedSplitterResult.addAll(Arrays.asList(0,4,8)); - assertEquals(expectedSplitterResult, splitterResult2); + expectedSplitterResult.addAll(Arrays.asList(0, 4, 8)); + assertEquals(expectedSplitterResult, splitterResultSink2.getSortedResult()); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java index bd97917750d77..cccec4045a874 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java @@ -24,12 +24,13 @@ import java.util.HashSet; import org.apache.flink.api.common.functions.CrossFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.function.sink.SinkFunction; import org.apache.flink.streaming.api.windowing.helper.Timestamp; +import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Test; @@ -39,17 +40,29 @@ public class WindowCrossJoinTest implements Serializable { private static final long MEMORYSIZE = 32; - private static ArrayList, Integer>> joinResults = new ArrayList, Integer>>(); private static ArrayList, Integer>> joinExpectedResults = new ArrayList, Integer>>(); - - private static ArrayList, Integer>> crossResults = new ArrayList, Integer>>(); private static ArrayList, Integer>> crossExpectedResults = new ArrayList, Integer>>(); + private static class MyTimestamp implements Timestamp { + + private static final long serialVersionUID = 1L; + + @Override + public long getTimestamp(T value) { + return 101L; + } + } + @Test public void test() throws Exception { StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE); env.setBufferTimeout(1); + TestListResultSink, Integer>> joinResultSink = + new TestListResultSink, Integer>>(); + TestListResultSink, Integer>> crossResultSink = + new TestListResultSink, Integer>>(); + ArrayList> in1 = new ArrayList>(); ArrayList> in2 = new ArrayList>(); @@ -101,7 +114,8 @@ public void test() throws Exception { .join(inStream2) .onWindow(1000, new MyTimestamp>(), new MyTimestamp>(), 100).where(0).equalTo(0) - .addSink(new JoinResultSink()); + .map(new ResultMap()) + .addSink(joinResultSink); inStream1 .cross(inStream2) @@ -116,50 +130,27 @@ public Tuple2, Tuple1> cross( Tuple2 val1, Tuple1 val2) throws Exception { return new Tuple2, Tuple1>(val1, val2); } - }).addSink(new CrossResultSink()); + }) + .map(new ResultMap()) + .addSink(crossResultSink); env.execute(); assertEquals(new HashSet, Integer>>(joinExpectedResults), - new HashSet, Integer>>(joinResults)); + new HashSet, Integer>>(joinResultSink.getResult())); assertEquals(new HashSet, Integer>>(crossExpectedResults), - new HashSet, Integer>>(crossResults)); - } - - private static class MyTimestamp implements Timestamp { - private static final long serialVersionUID = 1L; - - @Override - public long getTimestamp(T value) { - return 101L; - } + new HashSet, Integer>>(crossResultSink.getResult())); } - private static class JoinResultSink implements - SinkFunction, Tuple1>> { + private static class ResultMap implements + MapFunction, Tuple1>, + Tuple2, Integer>> { private static final long serialVersionUID = 1L; @Override - public void invoke(Tuple2, Tuple1> value) { - joinResults.add(new Tuple2, Integer>(value.f0, value.f1.f0)); - } - - @Override - public void cancel() { + public Tuple2, Integer> map(Tuple2, Tuple1> value) throws Exception { + return new Tuple2, Integer>(value.f0, value.f1.f0); } } - private static class CrossResultSink implements - SinkFunction, Tuple1>> { - private static final long serialVersionUID = 1L; - - @Override - public void invoke(Tuple2, Tuple1> value) { - crossResults.add(new Tuple2, Integer>(value.f0, value.f1.f0)); - } - - @Override - public void cancel() { - } - } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java index 9d166e56f808f..ffc7c7455b59f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java @@ -17,11 +17,7 @@ package org.apache.flink.streaming.api.collector; -import org.apache.flink.streaming.api.datastream.SplitDataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.function.sink.SinkFunction; -import org.apache.flink.streaming.util.TestStreamEnvironment; -import org.junit.Test; +import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.ArrayList; @@ -30,15 +26,17 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.assertEquals; +import org.apache.flink.streaming.api.datastream.SplitDataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.function.sink.SinkFunction; +import org.apache.flink.streaming.util.TestListResultSink; +import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.junit.Test; public class DirectedOutputTest { private static final String TEN = "ten"; private static final String ODD = "odd"; - private static final String ALL = "all"; - private static final String EVEN_AND_ODD = "evenAndOdd"; - private static final String ODD_AND_TEN = "oddAndTen"; private static final String EVEN = "even"; private static final String NON_SELECTED = "nonSelected"; @@ -98,19 +96,24 @@ public void cancel() { @Test public void outputSelectorTest() throws Exception { - StreamExecutionEnvironment env = new TestStreamEnvironment(1, 32); + StreamExecutionEnvironment env = new TestStreamEnvironment(3, 32); + + TestListResultSink evenSink = new TestListResultSink(); + TestListResultSink oddAndTenSink = new TestListResultSink(); + TestListResultSink evenAndOddSink = new TestListResultSink(); + TestListResultSink allSink = new TestListResultSink(); SplitDataStream source = env.generateSequence(1, 11).split(new MyOutputSelector()); - source.select(EVEN).addSink(new ListSink(EVEN)); - source.select(ODD, TEN).addSink(new ListSink(ODD_AND_TEN)); - source.select(EVEN, ODD).addSink(new ListSink(EVEN_AND_ODD)); - source.addSink(new ListSink(ALL)); + source.select(EVEN).addSink(evenSink); + source.select(ODD, TEN).addSink(oddAndTenSink); + source.select(EVEN, ODD).addSink(evenAndOddSink); + source.addSink(allSink); env.execute(); - assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), outputs.get(EVEN)); - assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L, 11L), outputs.get(ODD_AND_TEN)); + assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), evenSink.getSortedResult()); + assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L, 11L), oddAndTenSink.getSortedResult()); assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), - outputs.get(EVEN_AND_ODD)); - assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), outputs.get(ALL)); + evenAndOddSink.getResult()); + assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), allSink.getResult()); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java new file mode 100644 index 0000000000000..7f72173cdf17e --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.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.util; + + +import java.util.ArrayList; +import java.util.List; +import java.util.TreeSet; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.function.sink.RichSinkFunction; + +public class TestListResultSink extends RichSinkFunction { + + private int resultListId; + + public TestListResultSink() { + this.resultListId = TestListWrapper.getInstance().createList(); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + } + + @Override + public void invoke(T value) throws Exception { + synchronized (resultList()) { + resultList().add(value); + } + } + + @Override + public void close() throws Exception { + super.close(); + } + + private List resultList() { + synchronized (TestListWrapper.getInstance()) { + return (List) TestListWrapper.getInstance().getList(resultListId); + } + } + + public List getResult() { + synchronized (resultList()) { + ArrayList copiedList = new ArrayList(resultList()); + return copiedList; + } + } + + public List getSortedResult() { + synchronized (resultList()) { + TreeSet treeSet = new TreeSet(resultList()); + ArrayList sortedList = new ArrayList(treeSet); + return sortedList; + } + } + + @Override + public void cancel() { + + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java new file mode 100644 index 0000000000000..3c50f63009f0f --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.util; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class TestListWrapper { + + private static TestListWrapper instance; + + private List> lists; + + private TestListWrapper() { + lists = Collections.synchronizedList(new ArrayList>()); + } + + public static TestListWrapper getInstance() { + if (instance == null) { + instance = new TestListWrapper(); + } + return instance; + } + + /** + * Creates and stores a list, returns with the id. + * + * @return The ID of the list. + */ + public int createList() { + lists.add(new ArrayList()); + return lists.size() - 1; + } + + public List getList(int listId) { + List list = lists.get(listId); + if (list == null) { + throw new RuntimeException("No such list."); + } + + return list; + } + +} \ No newline at end of file From cf6feaedd4572b7f394acac05fbc9e62ae8b8a2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Thu, 26 Feb 2015 13:16:32 +0100 Subject: [PATCH 05/10] [FLINK-1594] [streaming] Added StreamGraphEdges --- .../flink/streaming/api/StreamEdge.java | 72 +++++++++++ .../flink/streaming/api/StreamEdgeList.java | 112 +++++++++++++++++ .../streaming/api/StreamEdgeListTest.java | 116 ++++++++++++++++++ 3 files changed, 300 insertions(+) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamEdgeListTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java new file mode 100644 index 0000000000000..8743233c49e95 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java @@ -0,0 +1,72 @@ +/* + * 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; + +import java.util.List; + +import org.apache.flink.streaming.partitioner.StreamPartitioner; + +public class StreamEdge { + + final private int sourceVertex; + final private int targetVertex; + final private int typeNumber; + final private List selectedNames; +// private OutputSelector outputSelector; + final private StreamPartitioner outputPartitioner; + + public StreamEdge(int sourceVertex, int targetVertex, int typeNumber, List selectedNames, StreamPartitioner outputPartitioner) { + this.sourceVertex = sourceVertex; + this.targetVertex = targetVertex; + this.typeNumber = typeNumber; + this.selectedNames = selectedNames; +// this.outputSelector = outputSelector; + this.outputPartitioner = outputPartitioner; + } + + public int getSourceVertex() { + return sourceVertex; + } + + public int getTargetVertex() { + return targetVertex; + } + + public int getTypeNumber() { + return typeNumber; + } + + public List getSelectedNames() { + return selectedNames; + } + + public StreamPartitioner getOutputPartitioner() { + return outputPartitioner; + } + + @Override + public String toString() { + return "StreamGraphEdge{" + + "sourceVertex=" + sourceVertex + + ", targetVertex=" + targetVertex + + ", typeNumber=" + typeNumber + + ", selectedNames=" + selectedNames + + ", outputPartitioner=" + outputPartitioner + + '}'; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java new file mode 100644 index 0000000000000..85202ab972e7a --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java @@ -0,0 +1,112 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class StreamEdgeList { + + private Map> outEdgeLists; + private Map> inEdgeLists; + + public StreamEdgeList() { + outEdgeLists = new HashMap>(); + inEdgeLists = new HashMap>(); + } + + public void addVertex(int vertexId) { + outEdgeLists.put(vertexId, new ArrayList()); + inEdgeLists.put(vertexId, new ArrayList()); + } + + public void removeVertex(int vertexId) { + ArrayList toRemove = new ArrayList(); + + for (StreamEdge edge : outEdgeLists.get(vertexId)) { + toRemove.add(edge); + } + + for (StreamEdge edge : inEdgeLists.get(vertexId)) { + toRemove.add(edge); + } + + for (StreamEdge edge : toRemove) { + removeEdge(edge); + } + + outEdgeLists.remove(vertexId); + inEdgeLists.remove(vertexId); + } + + public void addEdge(StreamEdge edge) { + int sourceId = edge.getSourceVertex(); + int targetId = edge.getTargetVertex(); + outEdgeLists.get(sourceId).add(edge); + inEdgeLists.get(targetId).add(edge); + } + + public void removeEdge(StreamEdge edge) { + int sourceId = edge.getSourceVertex(); + int targetId = edge.getTargetVertex(); + removeEdge(sourceId, targetId); + } + + public void removeEdge(int sourceId, int targetId) { + Iterator outIterator = outEdgeLists.get(sourceId).iterator(); + while (outIterator.hasNext()) { + StreamEdge edge = outIterator.next(); + + if (edge.getTargetVertex() == targetId) { + outIterator.remove(); + } + } + + Iterator inIterator = inEdgeLists.get(targetId).iterator(); + while (inIterator.hasNext()) { + StreamEdge edge = inIterator.next(); + + if (edge.getSourceVertex() == sourceId) { + inIterator.remove(); + } + } + } + + public List getOutEdges(int i) { + List outEdges = outEdgeLists.get(i); + + if (outEdges == null) { + throw new RuntimeException("No such vertex in stream graph: " + i); + } + + return outEdges; + } + + public List getInEdges(int i) { + List inEdges = inEdgeLists.get(i); + + if (inEdges == null) { + throw new RuntimeException("No such vertex in stream graph: " + i); + } + + return inEdges; + } +} \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamEdgeListTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamEdgeListTest.java new file mode 100644 index 0000000000000..c4ba987f89a92 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamEdgeListTest.java @@ -0,0 +1,116 @@ +/* + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; + +import org.junit.Before; +import org.junit.Test; + +public class StreamEdgeListTest { + + private StreamEdgeList edgeList; + + @Before + public void init() { + edgeList = new StreamEdgeList(); + } + + @Test + public void test() { + edgeList.addVertex(1); + edgeList.addVertex(2); + edgeList.addVertex(3); + + + // add edges + StreamEdge edge1 = new StreamEdge(1, 2, -1, null, null); + StreamEdge edge2 = new StreamEdge(2, 3, -1, null, null); + StreamEdge edge3 = new StreamEdge(1, 3, -1, null, null); + + edgeList.addEdge(edge1); + edgeList.addEdge(edge2); + edgeList.addEdge(edge3); + + // check adding + checkIfSameElements(edgeList.getOutEdges(1), Arrays.asList(edge1, edge3)); + checkIfSameElements(edgeList.getOutEdges(2), Arrays.asList(edge2)); + checkIfSameElements(edgeList.getOutEdges(3), new ArrayList()); + + checkIfSameElements(edgeList.getInEdges(1), new ArrayList()); + checkIfSameElements(edgeList.getInEdges(2), Arrays.asList(edge1)); + checkIfSameElements(edgeList.getInEdges(3), Arrays.asList(edge2, edge3)); + + // add duplicate edges + StreamEdge edge1new = new StreamEdge(1, 2, -2, null, null); + StreamEdge edge2new = new StreamEdge(2, 3, -2, null, null); + + edgeList.addEdge(edge1new); + edgeList.addEdge(edge2new); + + // check adding + checkIfSameElements(edgeList.getOutEdges(1), Arrays.asList(edge1, edge1new, edge3)); + checkIfSameElements(edgeList.getOutEdges(2), Arrays.asList(edge2, edge2new)); + checkIfSameElements(edgeList.getOutEdges(3), new ArrayList()); + + checkIfSameElements(edgeList.getInEdges(1), new ArrayList()); + checkIfSameElements(edgeList.getInEdges(2), Arrays.asList(edge1, edge1new)); + checkIfSameElements(edgeList.getInEdges(3), Arrays.asList(edge2, edge2new, edge3)); + + // remove a duplicate edge + edgeList.removeEdge(1, 2); + + // check removing + checkIfSameElements(edgeList.getOutEdges(1), Arrays.asList(edge3)); + checkIfSameElements(edgeList.getOutEdges(2), Arrays.asList(edge2, edge2new)); + + checkIfSameElements(edgeList.getInEdges(1), new ArrayList()); + checkIfSameElements(edgeList.getInEdges(2), new ArrayList()); + + // add back an edge and delete a vertex + edgeList.addEdge(edge1); + edgeList.removeVertex(2); + + // check removing + checkIfSameElements(edgeList.getOutEdges(1), Arrays.asList(edge3)); + try { + checkIfSameElements(edgeList.getOutEdges(2), null); + fail(); + } catch (RuntimeException e) { + } + checkIfSameElements(edgeList.getOutEdges(3), new ArrayList()); + + checkIfSameElements(edgeList.getInEdges(1), new ArrayList()); + try { + checkIfSameElements(edgeList.getInEdges(2), null); + fail(); + } catch (RuntimeException e) { + } + checkIfSameElements(edgeList.getInEdges(3), Arrays.asList(edge3)); + } + + private void checkIfSameElements(List expected, List result) { + assertEquals(new HashSet(expected), new HashSet(result)); + } +} \ No newline at end of file From 7f79fbb0c1b4a79a516733d8b8da08aa2a915cac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Thu, 26 Feb 2015 16:15:17 +0100 Subject: [PATCH 06/10] [FLINK-1594] [streaming] Embedded StreamEdges --- .../flink/streaming/api/StreamEdge.java | 2 +- .../flink/streaming/api/StreamEdgeList.java | 48 ++++++- .../flink/streaming/api/StreamGraph.java | 125 +++++++----------- .../api/StreamingJobGraphGenerator.java | 20 +-- .../streaming/api/WindowingOptimizer.java | 14 +- 5 files changed, 109 insertions(+), 100 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java index 8743233c49e95..479ae9342ed8c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java @@ -55,7 +55,7 @@ public List getSelectedNames() { return selectedNames; } - public StreamPartitioner getOutputPartitioner() { + public StreamPartitioner getPartitioner() { return outputPartitioner; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java index 85202ab972e7a..d15116bd8ae69 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdgeList.java @@ -90,23 +90,59 @@ public void removeEdge(int sourceId, int targetId) { } } - public List getOutEdges(int i) { - List outEdges = outEdgeLists.get(i); + public StreamEdge getEdge(int sourceId, int targetId) { + Iterator outIterator = outEdgeLists.get(sourceId).iterator(); + while (outIterator.hasNext()) { + StreamEdge edge = outIterator.next(); + + if (edge.getTargetVertex() == targetId) { + return edge; + } + } + + throw new RuntimeException("No such edge in stream graph: " + sourceId + " -> " + targetId); + } + + public List getOutEdges(int vertexId) { + List outEdges = outEdgeLists.get(vertexId); if (outEdges == null) { - throw new RuntimeException("No such vertex in stream graph: " + i); + throw new RuntimeException("No such vertex in stream graph: " + vertexId); } return outEdges; } - public List getInEdges(int i) { - List inEdges = inEdgeLists.get(i); + public List getInEdges(int vertexId) { + List inEdges = inEdgeLists.get(vertexId); if (inEdges == null) { - throw new RuntimeException("No such vertex in stream graph: " + i); + throw new RuntimeException("No such vertex in stream graph: " + vertexId); } return inEdges; } + + public List getOutEdgeIndices(int vertexId) { + List outEdges = getOutEdges(vertexId); + List outEdgeIndices = new ArrayList(); + + for (StreamEdge edge : outEdges) { + outEdgeIndices.add(edge.getTargetVertex()); + } + + return outEdgeIndices; + } + + public List getInEdgeIndices(int vertexId) { + List inEdges = getInEdges(vertexId); + + List inEdgeIndices = new ArrayList(); + + for (StreamEdge edge : inEdges) { + inEdgeIndices.add(edge.getSourceVertex()); + } + + return inEdgeIndices; + } } \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java index f0fbaabfc34ce..4d4d215467543 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java @@ -66,11 +66,11 @@ public class StreamGraph extends StreamingPlan { // Graph attributes private Map operatorParallelisms; private Map bufferTimeouts; - private Map> outEdgeLists; - private Map> outEdgeTypes; - private Map>> selectedNames; - private Map> inEdgeLists; - private Map>> outputPartitioners; + + private StreamEdgeList edges; + + private Map>> outputSelectors; + private Map operatorNames; private Map> invokableObjects; private Map> typeSerializersIn1; @@ -78,7 +78,6 @@ public class StreamGraph extends StreamingPlan { private Map> typeSerializersOut1; private Map> typeSerializersOut2; private Map> jobVertexClasses; - private Map>> outputSelectors; private Map iterationIds; private Map iterationIDtoHeadID; private Map iterationIDtoTailID; @@ -112,17 +111,9 @@ public void initGraph() { operatorParallelisms = new HashMap(); containingMaps.add(operatorParallelisms); bufferTimeouts = new HashMap(); - containingMaps.add(bufferTimeouts); - outEdgeLists = new HashMap>(); - containingMaps.add(outEdgeLists); - outEdgeTypes = new HashMap>(); - containingMaps.add(outEdgeTypes); - selectedNames = new HashMap>>(); - containingMaps.add(selectedNames); - inEdgeLists = new HashMap>(); - containingMaps.add(inEdgeLists); - outputPartitioners = new HashMap>>(); - containingMaps.add(outputPartitioners); + + edges = new StreamEdgeList(); + operatorNames = new HashMap(); containingMaps.add(operatorNames); invokableObjects = new HashMap>(); @@ -221,9 +212,10 @@ public void addIterationHead(Integer vertexID, Integer iterationHead, Integer it setSerializersFrom(iterationHead, vertexID); - setEdge(vertexID, iterationHead, - outputPartitioners.get(inEdgeLists.get(iterationHead).get(0)).get(0), 0, - new ArrayList()); + int outpartitionerIndexToCopy = edges.getInEdgeIndices(iterationHead).get(0); + StreamPartitioner outputPartitioner = edges.getOutEdges(outpartitionerIndexToCopy).get(0).getPartitioner(); + + setEdge(vertexID, iterationHead, outputPartitioner, 0, new ArrayList()); iterationTimeouts.put(iterationIDtoHeadID.get(iterationID), waitTime); @@ -290,7 +282,7 @@ public void addCoTask(Integer vertexID, /** * Sets vertex parameters in the JobGraph - * + * * @param vertexID * Name of the vertex * @param vertexClass @@ -307,12 +299,10 @@ private void addVertex(Integer vertexID, Class vert setParallelism(vertexID, parallelism); invokableObjects.put(vertexID, invokableObject); operatorNames.put(vertexID, operatorName); - outEdgeLists.put(vertexID, new ArrayList()); - outEdgeTypes.put(vertexID, new ArrayList()); - selectedNames.put(vertexID, new ArrayList>()); + + edges.addVertex(vertexID); outputSelectors.put(vertexID, new ArrayList>()); - inEdgeLists.put(vertexID, new ArrayList()); - outputPartitioners.put(vertexID, new ArrayList>()); + iterationTailCount.put(vertexID, 0); } @@ -333,40 +323,21 @@ private void addVertex(Integer vertexID, Class vert */ public void setEdge(Integer upStreamVertexID, Integer downStreamVertexID, StreamPartitioner partitionerObject, int typeNumber, List outputNames) { - outEdgeLists.get(upStreamVertexID).add(downStreamVertexID); - outEdgeTypes.get(upStreamVertexID).add(typeNumber); - inEdgeLists.get(downStreamVertexID).add(upStreamVertexID); - outputPartitioners.get(upStreamVertexID).add(partitionerObject); - selectedNames.get(upStreamVertexID).add(outputNames); + + StreamEdge edge = new StreamEdge(upStreamVertexID, downStreamVertexID, typeNumber, outputNames, partitionerObject); + edges.addEdge(edge); } public void removeEdge(Integer upStream, Integer downStream) { - int inputIndex = getInEdges(downStream).indexOf(upStream); - inEdgeLists.get(downStream).remove(inputIndex); - - int outputIndex = getOutEdges(upStream).indexOf(downStream); - outEdgeLists.get(upStream).remove(outputIndex); - outEdgeTypes.get(upStream).remove(outputIndex); - selectedNames.get(upStream).remove(outputIndex); - outputPartitioners.get(upStream).remove(outputIndex); + edges.removeEdge(upStream, downStream); } public void removeVertex(Integer toRemove) { - List outEdges = new ArrayList(getOutEdges(toRemove)); - List inEdges = new ArrayList(getInEdges(toRemove)); - - for (Integer output : outEdges) { - removeEdge(toRemove, output); - } - - for (Integer input : inEdges) { - removeEdge(input, toRemove); - } + edges.removeVertex(toRemove); for (Map map : containingMaps) { map.remove(toRemove); } - } private void addTypeSerializers(Integer vertexID, StreamRecordSerializer in1, @@ -418,11 +389,11 @@ public long getBufferTimeout(Integer vertexID) { /** * Sets a user defined {@link OutputSelector} for the given operator. Used * for directed emits. - * + * * @param vertexID - * Name of the vertex for which the output selector will be set + * Name of the vertex for which the output selector will be set * @param outputSelector - * The user defined output selector. + * The user defined output selector. */ public void setOutputSelector(Integer vertexID, OutputSelector outputSelector) { outputSelectors.get(vertexID).add(outputSelector); @@ -470,11 +441,11 @@ public StreamRecordSerializer getInSerializer2(Integer vertexID) { /** * Sets TypeSerializerWrapper from one vertex to another, used with some * sinks. - * + * * @param from - * from + * from * @param to - * to + * to */ public void setSerializersFrom(Integer from, Integer to) { operatorNames.put(to, operatorNames.get(from)); @@ -495,9 +466,9 @@ public JobGraph getJobGraph() { /** * Gets the assembled {@link JobGraph} and adds a user specified name for * it. - * + * * @param jobGraphName - * name of the jobGraph + * name of the jobGraph */ public JobGraph getJobGraph(String jobGraphName) { @@ -526,28 +497,24 @@ public Collection getSources() { return sources; } - public List getOutEdges(Integer vertexID) { - return outEdgeLists.get(vertexID); + public StreamEdge getEdge(Integer sourceId, Integer targetId) { + return edges.getEdge(sourceId, targetId); } - public List getInEdges(Integer vertexID) { - return inEdgeLists.get(vertexID); + public List getOutEdges(Integer vertexID) { + return edges.getOutEdges(vertexID); } - public List getOutEdgeTypes(Integer vertexID) { - - return outEdgeTypes.get(vertexID); + public List getInEdges(Integer vertexID) { + return edges.getInEdges(vertexID); } - public StreamPartitioner getOutPartitioner(Integer upStreamVertex, Integer downStreamVertex) { - return outputPartitioners.get(upStreamVertex).get( - outEdgeLists.get(upStreamVertex).indexOf(downStreamVertex)); + public List getOutEdgeIndices(Integer vertexID) { + return edges.getOutEdgeIndices(vertexID); } - public List getSelectedNames(Integer upStreamVertex, Integer downStreamVertex) { - - return selectedNames.get(upStreamVertex).get( - outEdgeLists.get(upStreamVertex).indexOf(downStreamVertex)); + public List getInEdgeIndices(Integer vertexID) { + return edges.getInEdgeIndices(vertexID); } public Collection getIterationIDs() { @@ -668,7 +635,9 @@ private void visit(JSONArray jsonArray, List toVisit, JSONArray inputs = new JSONArray(); node.put(PREDECESSORS, inputs); - for (int inputID : getInEdges(vertexID)) { + for (StreamEdge inEdge : getInEdges(vertexID)) { + int inputID = inEdge.getSourceVertex(); + Integer mappedID = (edgeRemapings.keySet().contains(inputID)) ? edgeRemapings.get(inputID) : inputID; decorateEdge(inputs, vertexID, mappedID, inputID); @@ -678,7 +647,9 @@ private void visit(JSONArray jsonArray, List toVisit, toVisit.remove(vertexID); } else { Integer iterationHead = -1; - for (int operator : getInEdges(vertexID)) { + for (StreamEdge inEdge : getInEdges(vertexID)) { + int operator = inEdge.getSourceVertex(); + if (iterationIds.keySet().contains(operator)) { iterationHead = operator; } @@ -718,7 +689,9 @@ private void visitIteration(JSONArray jsonArray, List toVisit, int head JSONArray inEdges = new JSONArray(); obj.put(PREDECESSORS, inEdges); - for (int inputID : getInEdges(vertexID)) { + for (StreamEdge inEdge : getInEdges(vertexID)) { + int inputID = inEdge.getSourceVertex(); + if (edgeRemapings.keySet().contains(inputID)) { decorateEdge(inEdges, vertexID, inputID, inputID); } else if (!iterationIds.containsKey(inputID)) { @@ -737,7 +710,7 @@ private void decorateEdge(JSONArray inputArray, int vertexID, int mappedInputID, JSONObject input = new JSONObject(); inputArray.put(input); input.put(ID, mappedInputID); - input.put(SHIP_STRATEGY, getOutPartitioner(inputID, vertexID).getStrategy()); + input.put(SHIP_STRATEGY, edges.getEdge(inputID, vertexID).getPartitioner().getStrategy()); input.put(SIDE, (inputArray.length() == 0) ? "first" : "second"); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java index ecb64553457a9..607d041901b48 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java @@ -104,11 +104,12 @@ private List> createChain(Integer startNode, Integer cu List chainableOutputs = new ArrayList(); List nonChainableOutputs = new ArrayList(); - for (Integer outName : streamGraph.getOutEdges(current)) { - if (isChainable(current, outName)) { - chainableOutputs.add(outName); + for (StreamEdge outEdge : streamGraph.getOutEdges(current)) { + Integer outID = outEdge.getTargetVertex(); + if (isChainable(current, outID)) { + chainableOutputs.add(outID); } else { - nonChainableOutputs.add(outName); + nonChainableOutputs.add(outID); } } @@ -230,7 +231,7 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, allOutputs.addAll(nonChainableOutputs); for (Integer output : allOutputs) { - config.setSelectedNames(output, streamGraph.getSelectedNames(vertexID, output)); + config.setSelectedNames(output, streamGraph.getEdge(vertexID, output).getSelectedNames()); } vertexConfigs.put(vertexID, config); @@ -251,14 +252,13 @@ private void connect(Integer headOfChain, Tuple2 edge) { headVertex.getConfiguration()) : chainedConfigs.get(headOfChain).get( upStreamvertexID); - List outEdgeIndexList = streamGraph.getOutEdgeTypes(upStreamvertexID); +// List outEdgeIndexList = streamGraph.getOutEdgeTypes(upStreamvertexID); int numOfInputs = downStreamConfig.getNumberOfInputs(); - downStreamConfig.setInputIndex(numOfInputs++, outEdgeIndexList.get(outputIndex)); + downStreamConfig.setInputIndex(numOfInputs++, streamGraph.getEdge(upStreamvertexID, downStreamvertexID).getTypeNumber()); downStreamConfig.setNumberOfInputs(numOfInputs); - StreamPartitioner partitioner = streamGraph.getOutPartitioner(upStreamvertexID, - downStreamvertexID); + StreamPartitioner partitioner = streamGraph.getEdge(upStreamvertexID, downStreamvertexID).getPartitioner(); upStreamConfig.setPartitioner(downStreamvertexID, partitioner); @@ -284,7 +284,7 @@ private boolean isChainable(Integer vertexID, Integer outName) { && outInvokable.getChainingStrategy() == ChainingStrategy.ALWAYS && (headInvokable.getChainingStrategy() == ChainingStrategy.HEAD || headInvokable .getChainingStrategy() == ChainingStrategy.ALWAYS) - && streamGraph.getOutPartitioner(vertexID, outName).getStrategy() == PartitioningStrategy.FORWARD + && streamGraph.getEdge(vertexID, outName).getPartitioner().getStrategy() == PartitioningStrategy.FORWARD && streamGraph.getParallelism(vertexID) == streamGraph.getParallelism(outName) && streamGraph.chaining; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/WindowingOptimizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/WindowingOptimizer.java index e2cbc4b6c95fc..3e98bdaec7b35 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/WindowingOptimizer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/WindowingOptimizer.java @@ -54,13 +54,13 @@ private static void removeMergeBeforeFlatten(StreamGraph streamGraph) { for (Integer flattener : flatteners) { // Flatteners should have exactly one input - Integer input = streamGraph.getInEdges(flattener).get(0); + Integer input = streamGraph.getInEdges(flattener).get(0).getSourceVertex(); // Check whether the flatten is applied after a merge if (streamGraph.getInvokable(input) instanceof WindowMerger) { // Mergers should have exactly one input - Integer mergeInput = streamGraph.getInEdges(input).get(0); + Integer mergeInput = streamGraph.getInEdges(input).get(0).getSourceVertex(); streamGraph.setEdge(mergeInput, flattener, new DistributePartitioner(true), 0, new ArrayList()); @@ -97,9 +97,9 @@ private static void setDiscretizerReuse(StreamGraph streamGraph) { boolean inMatching = false; for (Tuple2, List> matching : matchingDiscretizers) { Set discretizerInEdges = new HashSet( - streamGraph.getInEdges(discretizer.f0)); + streamGraph.getInEdgeIndices(discretizer.f0)); Set matchingInEdges = new HashSet( - streamGraph.getInEdges(matching.f1.get(0))); + streamGraph.getInEdgeIndices(matching.f1.get(0))); if (discretizer.f1.equals(matching.f0) && discretizerInEdges.equals(matchingInEdges)) { @@ -130,7 +130,7 @@ private static void setDiscretizerReuse(StreamGraph streamGraph) { private static void replaceDiscretizer(StreamGraph streamGraph, Integer toReplace, Integer replaceWith) { // Convert to array to create a copy - List outEdges = new ArrayList(streamGraph.getOutEdges(toReplace)); + List outEdges = new ArrayList(streamGraph.getOutEdgeIndices(toReplace)); int numOutputs = outEdges.size(); @@ -139,11 +139,11 @@ private static void replaceDiscretizer(StreamGraph streamGraph, Integer toReplac Integer output = outEdges.get(i); streamGraph.setEdge(replaceWith, output, - streamGraph.getOutPartitioner(toReplace, output), 0, new ArrayList()); + streamGraph.getEdge(toReplace, output).getPartitioner(), 0, new ArrayList()); streamGraph.removeEdge(toReplace, output); } - List inEdges = new ArrayList(streamGraph.getInEdges(toReplace)); + List inEdges = new ArrayList(streamGraph.getInEdgeIndices(toReplace)); // Remove inputs for (Integer input : inEdges) { streamGraph.removeEdge(input, toReplace); From a179d3714233f767a965e5b423a6fea5bc639c57 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Fri, 27 Feb 2015 11:40:01 +0100 Subject: [PATCH 07/10] [FLINK-1594] [streaming] Fixed co-tasks input handling --- .../flink/streaming/api/StreamConfig.java | 73 +++++++++++++------ .../flink/streaming/api/StreamEdge.java | 3 +- .../api/StreamingJobGraphGenerator.java | 43 ++++++----- .../api/streamvertex/CoStreamVertex.java | 7 +- .../api/streamvertex/OutputHandler.java | 8 +- 5 files changed, 89 insertions(+), 45 deletions(-) diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java index ea19a44f69c12..5b6de85a34f9c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java @@ -60,7 +60,11 @@ public class StreamConfig implements Serializable { private static final String TYPE_SERIALIZER_OUT_2 = "typeSerializer_out_2"; private static final String ITERATON_WAIT = "iterationWait"; private static final String OUTPUTS = "outvertexIDs"; + private static final String NONCHAINED_OUTPUTS = "NONCHAINED_OUTPUTS"; + private static final String CHAINED_OUTPUT_EDGES = "CHAINED_OUTPUTS"; private static final String EDGES_IN_ORDER = "rwOrder"; + private static final String OUT_STREAM_EDGES = "out stream edges"; + private static final String IN_STREAM_EDGES = "out stream edges"; // DEFAULT VALUES @@ -281,19 +285,60 @@ public int getNumberOfOutputs() { return config.getInteger(NUMBER_OF_OUTPUTS, 0); } - public void setOutputs(List outputvertexIDs) { - config.setBytes(OUTPUTS, SerializationUtils.serialize((Serializable) outputvertexIDs)); + public void setNonChainedOutputs(List outputvertexIDs) { + config.setBytes(NONCHAINED_OUTPUTS, SerializationUtils.serialize((Serializable) outputvertexIDs)); } @SuppressWarnings("unchecked") - public List getOutputs(ClassLoader cl) { + public List getNonChainedOutputs(ClassLoader cl) { try { - return (List) InstantiationUtil.readObjectFromConfig(this.config, OUTPUTS, cl); + return (List) InstantiationUtil.readObjectFromConfig(this.config, NONCHAINED_OUTPUTS, cl); } catch (Exception e) { throw new RuntimeException("Could not instantiate outputs."); } } + public void setChainedOutputs(List chainedOutputs) { + config.setBytes(CHAINED_OUTPUTS, + SerializationUtils.serialize((Serializable) chainedOutputs)); + } + + @SuppressWarnings("unchecked") + public List getChainedOutputs(ClassLoader cl) { + try { + return (List) InstantiationUtil.readObjectFromConfig(this.config, + CHAINED_OUTPUTS, cl); + } catch (Exception e) { + throw new RuntimeException("Could not instantiate chained outputs."); + } + } + + public void setOutEdges(List outEdges) { + config.setBytes(OUT_STREAM_EDGES, SerializationUtils.serialize((Serializable) outEdges)); + } + + public List getOutEdges(ClassLoader cl) { + try { + return (List) InstantiationUtil.readObjectFromConfig( + this.config, OUT_STREAM_EDGES, cl); + } catch (Exception e) { + throw new RuntimeException("Could not instantiate outputs."); + } + } + + public void setInEdges(List inEdges) { + config.setBytes(IN_STREAM_EDGES, SerializationUtils.serialize((Serializable) inEdges)); + } + + public List getInEdges(ClassLoader cl) { + try { + return (List) InstantiationUtil.readObjectFromConfig( + this.config, IN_STREAM_EDGES, cl); + } catch (Exception e) { + throw new RuntimeException("Could not instantiate inputs."); + } + } + public void setOutEdgesInOrder(List> outEdgeList) { config.setBytes(EDGES_IN_ORDER, SerializationUtils.serialize((Serializable) outEdgeList)); @@ -329,21 +374,6 @@ public int getInputIndex(int inputNumber) { return config.getInteger(INPUT_TYPE + inputNumber, 0); } - public void setChainedOutputs(List chainedOutputs) { - config.setBytes(CHAINED_OUTPUTS, - SerializationUtils.serialize((Serializable) chainedOutputs)); - } - - @SuppressWarnings("unchecked") - public List getChainedOutputs(ClassLoader cl) { - try { - return (List) InstantiationUtil.readObjectFromConfig(this.config, - CHAINED_OUTPUTS, cl); - } catch (Exception e) { - throw new RuntimeException("Could not instantiate chained outputs."); - } - } - public void setTransitiveChainedTaskConfigs(Map chainedTaskConfigs) { config.setBytes(CHAINED_TASK_CONFIG, SerializationUtils.serialize((Serializable) chainedTaskConfigs)); @@ -382,9 +412,10 @@ public String toString() { builder.append("\nTask name: " + getVertexID()); builder.append("\nNumber of non-chained inputs: " + getNumberOfInputs()); builder.append("\nNumber of non-chained outputs: " + getNumberOfOutputs()); - builder.append("\nOutput names: " + getOutputs(cl)); + builder.append("\nOutput names: " + getNonChainedOutputs(cl)); builder.append("\nPartitioning:"); - for (Integer outputname : getOutputs(cl)) { + for (StreamEdge output : getNonChainedOutputs(cl)) { + int outputname = output.getTargetVertex(); builder.append("\n\t" + outputname + ": " + getPartitioner(cl, outputname)); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java index 479ae9342ed8c..7363e08c90d0a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java @@ -17,11 +17,12 @@ package org.apache.flink.streaming.api; +import java.io.Serializable; import java.util.List; import org.apache.flink.streaming.partitioner.StreamPartitioner; -public class StreamEdge { +public class StreamEdge implements Serializable { final private int sourceVertex; final private int targetVertex; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java index 607d041901b48..3ff64cad2dc47 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java @@ -101,25 +101,24 @@ private List> createChain(Integer startNode, Integer cu if (!builtVertices.contains(startNode)) { List> transitiveOutEdges = new ArrayList>(); - List chainableOutputs = new ArrayList(); - List nonChainableOutputs = new ArrayList(); + List chainableOutputs = new ArrayList(); + List nonChainableOutputs = new ArrayList(); for (StreamEdge outEdge : streamGraph.getOutEdges(current)) { - Integer outID = outEdge.getTargetVertex(); - if (isChainable(current, outID)) { - chainableOutputs.add(outID); + if (isChainable(outEdge)) { + chainableOutputs.add(outEdge); } else { - nonChainableOutputs.add(outID); + nonChainableOutputs.add(outEdge); } } - for (Integer chainable : chainableOutputs) { - transitiveOutEdges.addAll(createChain(startNode, chainable)); + for (StreamEdge chainable : chainableOutputs) { + transitiveOutEdges.addAll(createChain(startNode, chainable.getTargetVertex())); } - for (Integer nonChainable : nonChainableOutputs) { - transitiveOutEdges.add(new Tuple2(current, nonChainable)); - createChain(nonChainable, nonChainable); + for (StreamEdge nonChainable : nonChainableOutputs) { + transitiveOutEdges.add(new Tuple2(current, nonChainable.getTargetVertex())); + createChain(nonChainable.getTargetVertex(), nonChainable.getTargetVertex()); } chainedNames.put(current, createChainedName(current, chainableOutputs)); @@ -133,6 +132,8 @@ private List> createChain(Integer startNode, Integer cu config.setChainStart(); config.setOutEdgesInOrder(transitiveOutEdges); + config.setOutEdges(streamGraph.getOutEdges(current)); + config.setInEdges(streamGraph.getInEdges(current)); for (Tuple2 edge : transitiveOutEdges) { connect(startNode, edge); @@ -157,12 +158,12 @@ private List> createChain(Integer startNode, Integer cu } } - private String createChainedName(Integer vertexID, List chainedOutputs) { + private String createChainedName(Integer vertexID, List chainedOutputs) { String operatorName = streamGraph.getOperatorName(vertexID); if (chainedOutputs.size() > 1) { List outputChainedNames = new ArrayList(); - for (Integer chainable : chainedOutputs) { - outputChainedNames.add(chainedNames.get(chainable)); + for (StreamEdge chainable : chainedOutputs) { + outputChainedNames.add(chainedNames.get(chainable.getTargetVertex())); } return operatorName + " -> (" + StringUtils.join(outputChainedNames, ", ") + ")"; } else if (chainedOutputs.size() == 1) { @@ -201,7 +202,7 @@ private StreamConfig createProcessingVertex(Integer vertexID) { } private void setVertexConfig(Integer vertexID, StreamConfig config, - List chainableOutputs, List nonChainableOutputs) { + List chainableOutputs, List nonChainableOutputs) { config.setVertexID(vertexID); config.setBufferTimeout(streamGraph.getBufferTimeout(vertexID)); @@ -215,7 +216,7 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, config.setOutputSelectors(streamGraph.getOutputSelector(vertexID)); config.setNumberOfOutputs(nonChainableOutputs.size()); - config.setOutputs(nonChainableOutputs); + config.setNonChainedOutputs(nonChainableOutputs); config.setChainedOutputs(chainableOutputs); config.setStateMonitoring(streamGraph.isMonitoringEnabled()); @@ -227,11 +228,11 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, config.setIterationWaitTime(streamGraph.getIterationTimeout(vertexID)); } - List allOutputs = new ArrayList(chainableOutputs); + List allOutputs = new ArrayList(chainableOutputs); allOutputs.addAll(nonChainableOutputs); - for (Integer output : allOutputs) { - config.setSelectedNames(output, streamGraph.getEdge(vertexID, output).getSelectedNames()); + for (StreamEdge output : allOutputs) { + config.setSelectedNames(output.getTargetVertex(), streamGraph.getEdge(vertexID, output.getTargetVertex()).getSelectedNames()); } vertexConfigs.put(vertexID, config); @@ -274,7 +275,9 @@ private void connect(Integer headOfChain, Tuple2 edge) { } } - private boolean isChainable(Integer vertexID, Integer outName) { + private boolean isChainable(StreamEdge edge) { + int vertexID = edge.getSourceVertex(); + int outName = edge.getTargetVertex(); StreamInvokable headInvokable = streamGraph.getInvokable(vertexID); StreamInvokable outInvokable = streamGraph.getInvokable(outName); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java index d794a35fd459c..b919e0faa623e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java @@ -20,6 +20,8 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.streaming.api.StreamEdge; +import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable; import org.apache.flink.streaming.api.streamrecord.StreamRecord; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.io.CoReaderIterator; @@ -29,6 +31,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.List; public class CoStreamVertex extends StreamVertex { @@ -78,8 +81,10 @@ protected void setConfigInputs() throws StreamVertexException { ArrayList inputList1 = new ArrayList(); ArrayList inputList2 = new ArrayList(); + List inEdges = configuration.getInEdges(userClassLoader); + for (int i = 0; i < numberOfInputs; i++) { - int inputType = configuration.getInputIndex(i); + int inputType = inEdges.get(i).getTypeNumber(); InputGate reader = getEnvironment().getInputGate(i); switch (inputType) { case 1: diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java index fd375f63d3e45..ca6b34d00e015 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.streaming.api.StreamConfig; +import org.apache.flink.streaming.api.StreamEdge; import org.apache.flink.streaming.api.collector.CollectorWrapper; import org.apache.flink.streaming.api.collector.DirectedCollectorWrapper; import org.apache.flink.streaming.api.collector.StreamOutput; @@ -117,7 +118,8 @@ private Collector createChainedCollector(StreamConfig chainedTaskConfig) { chainedTaskConfig.getOutputSelectors(cl)) : new CollectorWrapper(); // Create collectors for the network outputs - for (Integer output : chainedTaskConfig.getOutputs(cl)) { + for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(cl)) { + Integer output = outputEdge.getTargetVertex(); Collector outCollector = outputMap.get(output); @@ -130,7 +132,9 @@ private Collector createChainedCollector(StreamConfig chainedTaskConfig) { } // Create collectors for the chained outputs - for (Integer output : chainedTaskConfig.getChainedOutputs(cl)) { + for (StreamEdge outputEdge : chainedTaskConfig.getChainedOutputs(cl)) { + Integer output = outputEdge.getTargetVertex(); + Collector outCollector = createChainedCollector(chainedConfigs.get(output)); if (isDirectEmit) { ((DirectedCollectorWrapper) wrapper).addCollector(outCollector, From d452227e90e58f73ddd085bffc7625e253ef002b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Mon, 2 Mar 2015 10:20:14 +0100 Subject: [PATCH 08/10] [FLINK-1594] [streaming] Added OutputSelector wrapping --- .../flink/streaming/api/StreamConfig.java | 32 +++----- .../flink/streaming/api/StreamGraph.java | 8 +- .../api/StreamingJobGraphGenerator.java | 2 +- .../api/collector/CollectorWrapper.java | 17 ++--- .../BroadcastOutputSelectorWrapper.java | 43 +++++++++++ .../DirectedOutputSelectorWrapper.java} | 76 +++++-------------- .../{ => selector}/OutputSelector.java | 2 +- .../selector/OutputSelectorWrapper.java | 31 ++++++++ .../OutputSelectorWrapperFactory.java | 32 ++++++++ .../streaming/api/datastream/DataStream.java | 6 +- .../api/datastream/SplitDataStream.java | 2 +- .../api/streamvertex/OutputHandler.java | 39 ++++------ .../streaming/api/OutputSplitterTest.java | 2 +- .../api/collector/DirectedOutputTest.java | 1 + .../api/collector/OutputSelectorTest.java | 1 + .../examples/iteration/IterateExample.java | 2 +- .../streaming/api/scala/DataStream.scala | 2 +- 17 files changed, 175 insertions(+), 123 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java rename flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/{DirectedCollectorWrapper.java => selector/DirectedOutputSelectorWrapper.java} (59%) mode change 100755 => 100644 rename flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/{ => selector}/OutputSelector.java (96%) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java index 5b6de85a34f9c..f58f0addcb07a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java @@ -27,7 +27,7 @@ import org.apache.commons.lang3.SerializationUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.collector.OutputSelector; +import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.invokable.StreamInvokable; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.api.streamvertex.StreamVertexException; @@ -49,7 +49,8 @@ public class StreamConfig implements Serializable { private static final String VERTEX_NAME = "vertexID"; private static final String OPERATOR_NAME = "operatorName"; private static final String ITERATION_ID = "iteration-id"; - private static final String OUTPUT_SELECTOR = "outputSelector"; +// private static final String OUTPUT_SELECTOR = "outputSelector"; + private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper"; private static final String DIRECTED_EMIT = "directedEmit"; private static final String SERIALIZEDUDF = "serializedudf"; private static final String USER_FUNCTION = "userfunction"; @@ -67,7 +68,6 @@ public class StreamConfig implements Serializable { private static final String IN_STREAM_EDGES = "out stream edges"; // DEFAULT VALUES - private static final long DEFAULT_TIMEOUT = 100; public static final String STATE_MONITORING = "STATE_MONITORING"; @@ -189,33 +189,21 @@ public T getUserInvokable(ClassLoader cl) { } } - public void setDirectedEmit(boolean directedEmit) { - config.setBoolean(DIRECTED_EMIT, directedEmit); - } - - public boolean isDirectedEmit() { - return config.getBoolean(DIRECTED_EMIT, false); - } - - public void setOutputSelectors(List> outputSelector) { + public void setOutputSelectorWrapper(OutputSelectorWrapper outputSelectorWrapper) { try { - if (outputSelector != null && !outputSelector.isEmpty()) { - setDirectedEmit(true); - config.setBytes(OUTPUT_SELECTOR, - SerializationUtils.serialize((Serializable) outputSelector)); - } + config.setBytes(OUTPUT_SELECTOR_WRAPPER, SerializationUtils.serialize(outputSelectorWrapper)); } catch (SerializationException e) { - throw new RuntimeException("Cannot serialize OutputSelector"); + throw new RuntimeException("Cannot serialize OutputSelectorWrapper"); } } @SuppressWarnings("unchecked") - public List> getOutputSelectors(ClassLoader cl) { + public OutputSelectorWrapper getOutputSelectorWrapper(ClassLoader cl) { try { - return (List>) InstantiationUtil.readObjectFromConfig(this.config, - OUTPUT_SELECTOR, cl); + return (OutputSelectorWrapper) InstantiationUtil.readObjectFromConfig(this.config, + OUTPUT_SELECTOR_WRAPPER, cl); } catch (Exception e) { - throw new StreamVertexException("Cannot deserialize and instantiate OutputSelector", e); + throw new StreamVertexException("Cannot deserialize and instantiate OutputSelectorWrapper", e); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java index 4d4d215467543..6df170275fa3e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java @@ -37,7 +37,9 @@ import org.apache.flink.compiler.plan.StreamingPlan; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.streaming.api.collector.OutputSelector; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; +import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapperFactory; +import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.invokable.StreamInvokable; import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; @@ -537,8 +539,8 @@ public Class getJobVertexClass(Integer vertexID) { return inputFormatLists.get(vertexID); } - public List> getOutputSelector(Integer vertexID) { - return outputSelectors.get(vertexID); + public OutputSelectorWrapper getOutputSelectorWrapper(Integer vertexID) { + return OutputSelectorWrapperFactory.create(outputSelectors.get(vertexID)); } public Integer getIterationID(Integer vertexID) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java index 3ff64cad2dc47..79d43c086e421 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java @@ -213,7 +213,7 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, config.setTypeSerializerOut2(streamGraph.getOutSerializer2(vertexID)); config.setUserInvokable(streamGraph.getInvokable(vertexID)); - config.setOutputSelectors(streamGraph.getOutputSelector(vertexID)); + config.setOutputSelectorWrapper(streamGraph.getOutputSelectorWrapper(vertexID)); config.setNumberOfOutputs(nonChainableOutputs.size()); config.setNonChainedOutputs(nonChainableOutputs); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java index 1281bf07fbdc0..bb0268a49c3b6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java @@ -17,27 +17,26 @@ package org.apache.flink.streaming.api.collector; -import java.util.LinkedList; -import java.util.List; - +import org.apache.flink.streaming.api.StreamEdge; +import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.util.Collector; public class CollectorWrapper implements Collector { - private List> outputs; + private OutputSelectorWrapper outputSelectorWrapper; - public CollectorWrapper() { - this.outputs = new LinkedList>(); + public CollectorWrapper(OutputSelectorWrapper outputSelectorWrapper) { + this.outputSelectorWrapper = outputSelectorWrapper; } @SuppressWarnings("unchecked") - public void addCollector(Collector output) { - outputs.add((Collector) output); + public void addCollector(Collector output, StreamEdge edge) { + outputSelectorWrapper.addCollector(output, edge); } @Override public void collect(OUT record) { - for(Collector output: outputs){ + for (Collector output : outputSelectorWrapper.getSelectedOutputs(record)) { output.collect(record); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java new file mode 100644 index 0000000000000..44371f096496d --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java @@ -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.collector.selector; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.flink.streaming.api.StreamEdge; +import org.apache.flink.util.Collector; + +public class BroadcastOutputSelectorWrapper implements OutputSelectorWrapper { + + private List> outputs; + + public BroadcastOutputSelectorWrapper() { + outputs = new ArrayList>(); + } + + @Override + public void addCollector(Collector output, StreamEdge edge) { + outputs.add((Collector) output); + } + + @Override + public Iterable> getSelectedOutputs(OUT record) { + return outputs; + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedCollectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java old mode 100755 new mode 100644 similarity index 59% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedCollectorWrapper.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java index 4681cd34a8c8f..624fac1e904b5 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedCollectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.collector; +package org.apache.flink.streaming.api.collector.selector; -import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; @@ -25,49 +24,31 @@ import java.util.Map; import java.util.Set; +import org.apache.flink.streaming.api.StreamEdge; import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * A StreamCollector that uses user defined output names and a user defined - * output selector to make directed emits. - * - * @param - * Type of the Tuple collected. - */ -public class DirectedCollectorWrapper extends CollectorWrapper { - - private static final Logger LOG = LoggerFactory.getLogger(DirectedCollectorWrapper.class); +public class DirectedOutputSelectorWrapper implements OutputSelectorWrapper { - List> outputSelectors; + private static final Logger LOG = LoggerFactory.getLogger(DirectedOutputSelectorWrapper.class); - protected Map>> outputMap; + private List> outputSelectors; - private List> selectAllOutputs; - private Set> emitted; + private Map>> outputMap; + private Set> selectAllOutputs; +// private Set> emitted; - /** - * Creates a new DirectedStreamCollector - * - * @param outputSelector - * User defined {@link OutputSelector} - */ - public DirectedCollectorWrapper(List> outputSelectors) { + public DirectedOutputSelectorWrapper(List> outputSelectors) { this.outputSelectors = outputSelectors; - this.emitted = new HashSet>(); - this.selectAllOutputs = new LinkedList>(); +// this.emitted = new HashSet>(); + this.selectAllOutputs = new HashSet>(); //new LinkedList>(); this.outputMap = new HashMap>>(); - } @Override - public void addCollector(Collector output) { - addCollector(output, new ArrayList()); - } - - @SuppressWarnings("unchecked") - public void addCollector(Collector output, List selectedNames) { + public void addCollector(Collector output, StreamEdge edge) { + List selectedNames = edge.getSelectedNames(); if (selectedNames.isEmpty()) { selectAllOutputs.add((Collector) output); @@ -82,50 +63,33 @@ public void addCollector(Collector output, List selectedNames) { outputMap.get(selectedName).add((Collector) output); } } - } } } @Override - public void collect(OUT record) { - emitted.clear(); - - for (Collector output : selectAllOutputs) { - output.collect(record); - emitted.add(output); - } + public Iterable> getSelectedOutputs(OUT record) { + Set> selectedOutputs = new HashSet>(selectAllOutputs); for (OutputSelector outputSelector : outputSelectors) { Iterable outputNames = outputSelector.select(record); for (String outputName : outputNames) { List> outputList = outputMap.get(outputName); - if (outputList == null) { + + try { + selectedOutputs.addAll(outputList); + } catch (NullPointerException e) { if (LOG.isErrorEnabled()) { String format = String.format( "Cannot emit because no output is selected with the name: %s", outputName); LOG.error(format); - - } - } else { - for (Collector output : outputList) { - if (!emitted.contains(output)) { - output.collect(record); - emitted.add(output); - } } - } - } } - } - - @Override - public void close() { - + return selectedOutputs; } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java similarity index 96% rename from flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java rename to flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java index 6dbcff44a2410..b886fa627e69a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/OutputSelector.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.collector; +package org.apache.flink.streaming.api.collector.selector; import java.io.Serializable; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java new file mode 100644 index 0000000000000..850a1d984bdfb --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.collector.selector; + +import java.io.Serializable; + +import org.apache.flink.streaming.api.StreamEdge; +import org.apache.flink.util.Collector; + +public interface OutputSelectorWrapper extends Serializable { + + public void addCollector(Collector output, StreamEdge edge); + + public Iterable> getSelectedOutputs(OUT record); + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java new file mode 100644 index 0000000000000..c0f22c7a0cfc3 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.collector.selector; + +import java.util.List; + +public class OutputSelectorWrapperFactory { + + public static OutputSelectorWrapper create(List> outputSelectors) { + if (outputSelectors.size() == 0) { + return new BroadcastOutputSelectorWrapper(); + } else { + return new DirectedOutputSelectorWrapper(outputSelectors); + } + } + +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 879a98c04f65c..5f6f981fee6fc 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -47,7 +47,7 @@ import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.core.fs.Path; import org.apache.flink.streaming.api.StreamGraph; -import org.apache.flink.streaming.api.collector.OutputSelector; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.temporaloperator.StreamCrossOperator; import org.apache.flink.streaming.api.datastream.temporaloperator.StreamJoinOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -238,12 +238,12 @@ public DataStream merge(DataStream... streams) { /** * Operator used for directing tuples to specific named outputs using an - * {@link org.apache.flink.streaming.api.collector.OutputSelector}. Calling + * {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}. Calling * this method on an operator creates a new {@link SplitDataStream}. * * @param outputSelector * The user defined - * {@link org.apache.flink.streaming.api.collector.OutputSelector} + * {@link org.apache.flink.streaming.api.collector.selector.OutputSelector} * for directing the tuples. * @return The {@link SplitDataStream} */ diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java index 97458a8d99b83..69e059e327142 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java @@ -19,7 +19,7 @@ import java.util.Arrays; -import org.apache.flink.streaming.api.collector.OutputSelector; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; /** * The SplitDataStream represents an operator that has been split using an diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java index ca6b34d00e015..18ddc798ab594 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java @@ -30,8 +30,8 @@ import org.apache.flink.streaming.api.StreamConfig; import org.apache.flink.streaming.api.StreamEdge; import org.apache.flink.streaming.api.collector.CollectorWrapper; -import org.apache.flink.streaming.api.collector.DirectedCollectorWrapper; import org.apache.flink.streaming.api.collector.StreamOutput; +import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.invokable.ChainableInvokable; import org.apache.flink.streaming.api.streamrecord.StreamRecord; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; @@ -100,22 +100,22 @@ public Collection> getOutputs() { * This method builds up a nested collector which encapsulates all the * chained operators and their network output. The result of this recursive * call will be passed as collector to the first invokable in the chain. - * + * * @param chainedTaskConfig - * The configuration of the starting operator of the chain, we - * use this paramater to recursively build the whole chain + * The configuration of the starting operator of the chain, we + * use this paramater to recursively build the whole chain * @return Returns the collector for the chain starting from the given - * config + * config */ - @SuppressWarnings({ "unchecked", "rawtypes" }) + @SuppressWarnings({"unchecked", "rawtypes"}) private Collector createChainedCollector(StreamConfig chainedTaskConfig) { - boolean isDirectEmit = chainedTaskConfig.isDirectedEmit(); // We create a wrapper that will encapsulate the chained operators and // network outputs - CollectorWrapper wrapper = isDirectEmit ? new DirectedCollectorWrapper( - chainedTaskConfig.getOutputSelectors(cl)) : new CollectorWrapper(); + + OutputSelectorWrapper outputSelectorWrapper = chainedTaskConfig.getOutputSelectorWrapper(cl); + CollectorWrapper wrapper = new CollectorWrapper(outputSelectorWrapper); // Create collectors for the network outputs for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(cl)) { @@ -123,12 +123,7 @@ private Collector createChainedCollector(StreamConfig chainedTaskConfig) { Collector outCollector = outputMap.get(output); - if (isDirectEmit) { - ((DirectedCollectorWrapper) wrapper).addCollector(outCollector, - chainedTaskConfig.getSelectedNames(output)); - } else { - wrapper.addCollector(outCollector); - } + wrapper.addCollector(outCollector, outputEdge); } // Create collectors for the chained outputs @@ -136,12 +131,8 @@ private Collector createChainedCollector(StreamConfig chainedTaskConfig) { Integer output = outputEdge.getTargetVertex(); Collector outCollector = createChainedCollector(chainedConfigs.get(output)); - if (isDirectEmit) { - ((DirectedCollectorWrapper) wrapper).addCollector(outCollector, - chainedTaskConfig.getSelectedNames(output)); - } else { - wrapper.addCollector(outCollector); - } + + wrapper.addCollector(outCollector, outputEdge); } if (chainedTaskConfig.isChainStart()) { @@ -169,11 +160,11 @@ public Collector getCollector() { /** * We create the StreamOutput for the specific output given by the id, and * the configuration of its source task - * + * * @param outputVertex - * Name of the output to which the streamoutput will be set up + * Name of the output to which the streamoutput will be set up * @param configuration - * The config of upStream task + * The config of upStream task * @return */ private StreamOutput createStreamOutput(Integer outputVertex, diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java index cf6bb3cacfe35..14f0fa04af103 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java @@ -23,7 +23,7 @@ import java.util.Arrays; import java.util.List; -import org.apache.flink.streaming.api.collector.OutputSelector; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.util.TestListResultSink; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java index ffc7c7455b59f..13bf457062c2c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.SplitDataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.function.sink.SinkFunction; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java index 1615a45f0d825..a3d89f271e6fc 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.junit.Test; public class OutputSelectorTest { diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java index 998e81842ddd1..bbd543309cc6f 100644 --- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java +++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java @@ -25,7 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.collector.OutputSelector; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.IterativeDataStream; import org.apache.flink.streaming.api.datastream.SplitDataStream; diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 59b19068acf2a..3dc54d62645da 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.scala import org.apache.flink.api.java.typeutils.TupleTypeInfoBase +import org.apache.flink.streaming.api.collector.selector.OutputSelector import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, SingleOutputStreamOperator, GroupedDataStream} import scala.collection.JavaConverters._ @@ -38,7 +39,6 @@ import org.apache.flink.streaming.api.function.sink.SinkFunction import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.clean import org.apache.flink.streaming.api.windowing.helper.WindowingHelper import org.apache.flink.streaming.api.windowing.policy.{ EvictionPolicy, TriggerPolicy } -import org.apache.flink.streaming.api.collector.OutputSelector import scala.collection.JavaConversions._ import java.util.HashMap import org.apache.flink.streaming.api.function.aggregation.SumFunction From 38d69cf5b1cfc42ba57887999b95391336d533df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Thu, 5 Mar 2015 15:51:51 +0100 Subject: [PATCH 09/10] [streaming] Added Reader and Writer factories & minor cleanup --- .../flink/streaming/api/StreamConfig.java | 8 +-- .../flink/streaming/api/StreamEdge.java | 14 ++++- .../flink/streaming/api/StreamEdgeList.java | 4 ++ .../api/StreamingJobGraphGenerator.java | 1 - .../DirectedOutputSelectorWrapper.java | 2 - .../api/streamvertex/CoStreamVertex.java | 58 +++++++++---------- .../api/streamvertex/InputHandler.java | 13 +++-- .../api/streamvertex/OutputHandler.java | 25 ++------ .../flink/streaming/io/InputGateFactory.java | 42 ++++++++++++++ .../streaming/io/RecordWriterFactory.java | 52 +++++++++++++++++ 10 files changed, 153 insertions(+), 66 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/InputGateFactory.java create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/RecordWriterFactory.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java index f58f0addcb07a..fc1f43584abf0 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java @@ -49,9 +49,7 @@ public class StreamConfig implements Serializable { private static final String VERTEX_NAME = "vertexID"; private static final String OPERATOR_NAME = "operatorName"; private static final String ITERATION_ID = "iteration-id"; -// private static final String OUTPUT_SELECTOR = "outputSelector"; private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper"; - private static final String DIRECTED_EMIT = "directedEmit"; private static final String SERIALIZEDUDF = "serializedudf"; private static final String USER_FUNCTION = "userfunction"; private static final String BUFFER_TIMEOUT = "bufferTimeout"; @@ -60,9 +58,7 @@ public class StreamConfig implements Serializable { private static final String TYPE_SERIALIZER_OUT_1 = "typeSerializer_out_1"; private static final String TYPE_SERIALIZER_OUT_2 = "typeSerializer_out_2"; private static final String ITERATON_WAIT = "iterationWait"; - private static final String OUTPUTS = "outvertexIDs"; private static final String NONCHAINED_OUTPUTS = "NONCHAINED_OUTPUTS"; - private static final String CHAINED_OUTPUT_EDGES = "CHAINED_OUTPUTS"; private static final String EDGES_IN_ORDER = "rwOrder"; private static final String OUT_STREAM_EDGES = "out stream edges"; private static final String IN_STREAM_EDGES = "out stream edges"; @@ -247,7 +243,7 @@ public void setSelectedNames(Integer output, List selected) { SerializationUtils.serialize((Serializable) selected)); } else { config.setBytes(OUTPUT_NAME + output, - SerializationUtils.serialize((Serializable) new ArrayList())); + SerializationUtils.serialize(new ArrayList())); } } @@ -305,6 +301,7 @@ public void setOutEdges(List outEdges) { config.setBytes(OUT_STREAM_EDGES, SerializationUtils.serialize((Serializable) outEdges)); } + @SuppressWarnings("unchecked") public List getOutEdges(ClassLoader cl) { try { return (List) InstantiationUtil.readObjectFromConfig( @@ -318,6 +315,7 @@ public void setInEdges(List inEdges) { config.setBytes(IN_STREAM_EDGES, SerializationUtils.serialize((Serializable) inEdges)); } + @SuppressWarnings("unchecked") public List getInEdges(ClassLoader cl) { try { return (List) InstantiationUtil.readObjectFromConfig( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java index 7363e08c90d0a..a300fc2585feb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java @@ -22,13 +22,24 @@ import org.apache.flink.streaming.partitioner.StreamPartitioner; +/** + * An edge in the streaming topology. One edge like this does not necessarily + * gets converted to a connection between two job vertices (due to chaining/optimization). + */ public class StreamEdge implements Serializable { final private int sourceVertex; final private int targetVertex; + + /** + * The type number of the input for co-tasks. + */ final private int typeNumber; + + /** + * A list of output names that the target vertex listens to (if there is output selection). + */ final private List selectedNames; -// private OutputSelector outputSelector; final private StreamPartitioner outputPartitioner; public StreamEdge(int sourceVertex, int targetVertex, int typeNumber, List selectedNames, StreamPartitioner outputPartitioner) { @@ -36,7 +47,6 @@ public StreamEdge(int sourceVertex, int targetVertex, int typeNumber, List> outEdgeLists; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java index 79d43c086e421..d42003049bb63 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java @@ -253,7 +253,6 @@ private void connect(Integer headOfChain, Tuple2 edge) { headVertex.getConfiguration()) : chainedConfigs.get(headOfChain).get( upStreamvertexID); -// List outEdgeIndexList = streamGraph.getOutEdgeTypes(upStreamvertexID); int numOfInputs = downStreamConfig.getNumberOfInputs(); downStreamConfig.setInputIndex(numOfInputs++, streamGraph.getEdge(upStreamvertexID, downStreamvertexID).getTypeNumber()); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java index 624fac1e904b5..a04900c2eab11 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java @@ -37,11 +37,9 @@ public class DirectedOutputSelectorWrapper implements OutputSelectorWrapper private Map>> outputMap; private Set> selectAllOutputs; -// private Set> emitted; public DirectedOutputSelectorWrapper(List> outputSelectors) { this.outputSelectors = outputSelectors; -// this.emitted = new HashSet>(); this.selectAllOutputs = new HashSet>(); //new LinkedList>(); this.outputMap = new HashMap>>(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java index b919e0faa623e..b45fc4436ab76 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java @@ -17,22 +17,21 @@ package org.apache.flink.streaming.api.streamvertex; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.streaming.api.StreamEdge; -import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable; import org.apache.flink.streaming.api.streamrecord.StreamRecord; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.io.CoReaderIterator; import org.apache.flink.streaming.io.CoRecordReader; import org.apache.flink.streaming.io.IndexedReaderIterator; +import org.apache.flink.streaming.io.InputGateFactory; import org.apache.flink.util.MutableObjectIterator; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - public class CoStreamVertex extends StreamVertex { protected StreamRecordSerializer inputDeserializer1 = null; @@ -87,22 +86,19 @@ protected void setConfigInputs() throws StreamVertexException { int inputType = inEdges.get(i).getTypeNumber(); InputGate reader = getEnvironment().getInputGate(i); switch (inputType) { - case 1: - inputList1.add(reader); - break; - case 2: - inputList2.add(reader); - break; - default: - throw new RuntimeException("Invalid input type number: " + inputType); + case 1: + inputList1.add(reader); + break; + case 2: + inputList2.add(reader); + break; + default: + throw new RuntimeException("Invalid input type number: " + inputType); } } - final InputGate reader1 = inputList1.size() == 1 ? inputList1.get(0) : new UnionInputGate( - inputList1.toArray(new InputGate[inputList1.size()])); - - final InputGate reader2 = inputList2.size() == 1 ? inputList2.get(0) : new UnionInputGate( - inputList2.toArray(new InputGate[inputList2.size()])); + final InputGate reader1 = InputGateFactory.createInputGate(inputList1); + final InputGate reader2 = InputGateFactory.createInputGate(inputList2); coReader = new CoRecordReader>, DeserializationDelegate>>( reader1, reader2); @@ -112,12 +108,12 @@ protected void setConfigInputs() throws StreamVertexException { @Override public MutableObjectIterator getInput(int index) { switch (index) { - case 0: - return (MutableObjectIterator) inputIter1; - case 1: - return (MutableObjectIterator) inputIter2; - default: - throw new IllegalArgumentException("CoStreamVertex has only 2 inputs"); + case 0: + return (MutableObjectIterator) inputIter1; + case 1: + return (MutableObjectIterator) inputIter2; + default: + throw new IllegalArgumentException("CoStreamVertex has only 2 inputs"); } } @@ -130,12 +126,12 @@ public IndexedReaderIterator getIndexedInput(int index) { @Override public StreamRecordSerializer getInputSerializer(int index) { switch (index) { - case 0: - return (StreamRecordSerializer) inputDeserializer1; - case 1: - return (StreamRecordSerializer) inputDeserializer2; - default: - throw new IllegalArgumentException("CoStreamVertex has only 2 inputs"); + case 0: + return (StreamRecordSerializer) inputDeserializer1; + case 1: + return (StreamRecordSerializer) inputDeserializer2; + default: + throw new IllegalArgumentException("CoStreamVertex has only 2 inputs"); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java index 48a00d8708c8a..3988ec18f5bb2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java @@ -22,13 +22,13 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.io.network.api.reader.MutableReader; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.streaming.api.StreamConfig; import org.apache.flink.streaming.api.streamrecord.StreamRecord; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.io.IndexedMutableReader; import org.apache.flink.streaming.io.IndexedReaderIterator; +import org.apache.flink.streaming.io.InputGateFactory; public class InputHandler { private StreamRecordSerializer inputSerializer = null; @@ -56,17 +56,20 @@ protected void setConfigInputs() throws StreamVertexException { int numberOfInputs = configuration.getNumberOfInputs(); if (numberOfInputs > 0) { - InputGate inputGate = numberOfInputs < 2 ? streamVertex.getEnvironment() - .getInputGate(0) : new UnionInputGate(streamVertex.getEnvironment() - .getAllInputGates()); - + InputGate inputGate = InputGateFactory.createInputGate(streamVertex.getEnvironment().getAllInputGates()); inputs = new IndexedMutableReader>>(inputGate); + inputs.registerTaskEventListener(streamVertex.getSuperstepListener(), StreamingSuperstep.class); inputIter = new IndexedReaderIterator>(inputs, inputSerializer); } + } + private IndexedReaderIterator> createInputIterator() { + final IndexedReaderIterator> iter = new IndexedReaderIterator>( + inputs, inputSerializer); + return iter; } protected static IndexedReaderIterator> staticCreateInputIterator( diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java index 18ddc798ab594..bb66c63ffa8bb 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java @@ -26,6 +26,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.streaming.api.StreamConfig; import org.apache.flink.streaming.api.StreamEdge; @@ -35,7 +36,7 @@ import org.apache.flink.streaming.api.invokable.ChainableInvokable; import org.apache.flink.streaming.api.streamrecord.StreamRecord; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; -import org.apache.flink.streaming.io.StreamRecordWriter; +import org.apache.flink.streaming.io.RecordWriterFactory; import org.apache.flink.streaming.partitioner.StreamPartitioner; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -181,26 +182,10 @@ private StreamOutput createStreamOutput(Integer outputVertex, StreamPartitioner outputPartitioner = configuration.getPartitioner(cl, outputVertex); - RecordWriter>> output; + ResultPartitionWriter bufferWriter = vertex.getEnvironment().getWriter(outputIndex); - if (configuration.getBufferTimeout() >= 0) { - - output = new StreamRecordWriter>>(vertex - .getEnvironment().getWriter(outputIndex), outputPartitioner, - configuration.getBufferTimeout()); - - if (LOG.isTraceEnabled()) { - LOG.trace("StreamRecordWriter initiated with {} bufferTimeout for {}", - configuration.getBufferTimeout(), vertex.getClass().getSimpleName()); - } - } else { - output = new RecordWriter>>(vertex - .getEnvironment().getWriter(outputIndex), outputPartitioner); - - if (LOG.isTraceEnabled()) { - LOG.trace("RecordWriter initiated for {}", vertex.getClass().getSimpleName()); - } - } + RecordWriter>> output = + RecordWriterFactory.createRecordWriter(bufferWriter, outputPartitioner, configuration.getBufferTimeout()); StreamOutput streamOutput = new StreamOutput(output, vertex.instanceID, outSerializationDelegate); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/InputGateFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/InputGateFactory.java new file mode 100644 index 0000000000000..3e6edb978ef76 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/InputGateFactory.java @@ -0,0 +1,42 @@ +/* + * 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.io; + +import java.util.Collection; + +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; + +public class InputGateFactory { + + public static InputGate createInputGate(Collection inputGates) { + return createInputGate(inputGates.toArray(new InputGate[inputGates.size()])); + } + + public static InputGate createInputGate(InputGate[] inputGates) { + if (inputGates.length <= 0) { + throw new RuntimeException("No such input gate."); + } + + if (inputGates.length < 2) { + return inputGates[0]; + } else { + return new UnionInputGate(inputGates); + } + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/RecordWriterFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/RecordWriterFactory.java new file mode 100644 index 0000000000000..e859225658bfa --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/RecordWriterFactory.java @@ -0,0 +1,52 @@ +/* + * 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.io; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.io.network.api.writer.ChannelSelector; +import org.apache.flink.runtime.io.network.api.writer.RecordWriter; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RecordWriterFactory { + private static final Logger LOG = LoggerFactory.getLogger(RecordWriterFactory.class); + + public static RecordWriter createRecordWriter(ResultPartitionWriter bufferWriter, ChannelSelector channelSelector, long bufferTimeout) { + + RecordWriter output; + + if (bufferTimeout >= 0) { + output = new StreamRecordWriter(bufferWriter, channelSelector, bufferTimeout); + + if (LOG.isTraceEnabled()) { + LOG.trace("StreamRecordWriter initiated with {} bufferTimeout.", bufferTimeout); + } + } else { + output = new RecordWriter(bufferWriter, channelSelector); + + if (LOG.isTraceEnabled()) { + LOG.trace("RecordWriter initiated."); + } + } + + return output; + + } + +} From b1f93a11ba4bfc2660556171b9da727a896a4b15 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?G=C3=A1bor=20Hermann?= Date: Thu, 12 Mar 2015 08:12:21 +0100 Subject: [PATCH 10/10] [FLINK-1594] [streaming] Self connect tests and fixes --- .../flink/streaming/api/StreamConfig.java | 56 +--- .../flink/streaming/api/StreamEdge.java | 41 ++- .../api/StreamingJobGraphGenerator.java | 88 +++--- .../api/collector/CollectorWrapper.java | 1 - .../BroadcastOutputSelectorWrapper.java | 2 + .../DirectedOutputSelectorWrapper.java | 3 + .../OutputSelectorWrapperFactory.java | 1 + .../api/streamvertex/CoStreamVertex.java | 2 +- .../api/streamvertex/InputHandler.java | 6 - .../api/streamvertex/OutputHandler.java | 39 +-- .../flink/streaming/api/CoStreamTest.java | 92 +++++-- .../operator/co/SelfConnectionTest.java | 252 ++++++++++++++++++ .../api/streamvertex/StreamVertexTest.java | 2 + .../streaming/util/TestListResultSink.java | 2 + .../flink/streaming/util/TestListWrapper.java | 4 + 15 files changed, 455 insertions(+), 136 deletions(-) create mode 100644 flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/co/SelfConnectionTest.java diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java index fc1f43584abf0..c1e9606dea767 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java @@ -25,27 +25,23 @@ import org.apache.commons.lang3.SerializationException; import org.apache.commons.lang3.SerializationUtils; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper; import org.apache.flink.streaming.api.invokable.StreamInvokable; import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.api.streamvertex.StreamVertexException; -import org.apache.flink.streaming.partitioner.StreamPartitioner; import org.apache.flink.util.InstantiationUtil; public class StreamConfig implements Serializable { private static final long serialVersionUID = 1L; - private static final String INPUT_TYPE = "inputType_"; private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs"; private static final String NUMBER_OF_INPUTS = "numberOfInputs"; private static final String CHAINED_OUTPUTS = "chainedOutputs"; private static final String CHAINED_TASK_CONFIG = "chainedTaskConfig_"; private static final String IS_CHAINED_VERTEX = "isChainedSubtask"; private static final String OUTPUT_NAME = "outputName_"; - private static final String PARTITIONER_OBJECT = "partitionerObject_"; private static final String VERTEX_NAME = "vertexID"; private static final String OPERATOR_NAME = "operatorName"; private static final String ITERATION_ID = "iteration-id"; @@ -219,24 +215,6 @@ public long getIterationWaitTime() { return config.getLong(ITERATON_WAIT, 0); } - public void setPartitioner(Integer output, StreamPartitioner partitionerObject) { - - config.setBytes(PARTITIONER_OBJECT + output, - SerializationUtils.serialize(partitionerObject)); - } - - @SuppressWarnings("unchecked") - public StreamPartitioner getPartitioner(ClassLoader cl, Integer output) { - StreamPartitioner partitioner = null; - try { - partitioner = (StreamPartitioner) InstantiationUtil.readObjectFromConfig( - this.config, PARTITIONER_OBJECT + output, cl); - } catch (Exception e) { - throw new RuntimeException("Partitioner could not be instantiated."); - } - return partitioner; - } - public void setSelectedNames(Integer output, List selected) { if (selected != null) { config.setBytes(OUTPUT_NAME + output, @@ -311,12 +289,12 @@ public List getOutEdges(ClassLoader cl) { } } - public void setInEdges(List inEdges) { + public void setInPhysicalEdges(List inEdges) { config.setBytes(IN_STREAM_EDGES, SerializationUtils.serialize((Serializable) inEdges)); } @SuppressWarnings("unchecked") - public List getInEdges(ClassLoader cl) { + public List getInPhysicalEdges(ClassLoader cl) { try { return (List) InstantiationUtil.readObjectFromConfig( this.config, IN_STREAM_EDGES, cl); @@ -325,41 +303,31 @@ public List getInEdges(ClassLoader cl) { } } - public void setOutEdgesInOrder(List> outEdgeList) { - - config.setBytes(EDGES_IN_ORDER, SerializationUtils.serialize((Serializable) outEdgeList)); - } - - public void setStateMonitoring(boolean stateMonitoring) { - + config.setBoolean(STATE_MONITORING, stateMonitoring); - + } - + public boolean getStateMonitoring() { return config.getBoolean(STATE_MONITORING, false); } + public void setOutEdgesInOrder(List outEdgeList) { + config.setBytes(EDGES_IN_ORDER, SerializationUtils.serialize((Serializable) outEdgeList)); + } + @SuppressWarnings("unchecked") - public List> getOutEdgesInOrder(ClassLoader cl) { + public List getOutEdgesInOrder(ClassLoader cl) { try { - return (List>) InstantiationUtil.readObjectFromConfig( + return (List) InstantiationUtil.readObjectFromConfig( this.config, EDGES_IN_ORDER, cl); } catch (Exception e) { throw new RuntimeException("Could not instantiate outputs."); } } - public void setInputIndex(int inputNumber, Integer inputTypeNumber) { - config.setInteger(INPUT_TYPE + inputNumber++, inputTypeNumber); - } - - public int getInputIndex(int inputNumber) { - return config.getInteger(INPUT_TYPE + inputNumber, 0); - } - public void setTransitiveChainedTaskConfigs(Map chainedTaskConfigs) { config.setBytes(CHAINED_TASK_CONFIG, SerializationUtils.serialize((Serializable) chainedTaskConfigs)); @@ -402,7 +370,7 @@ public String toString() { builder.append("\nPartitioning:"); for (StreamEdge output : getNonChainedOutputs(cl)) { int outputname = output.getTargetVertex(); - builder.append("\n\t" + outputname + ": " + getPartitioner(cl, outputname)); + builder.append("\n\t" + outputname + ": " + output.getPartitioner()); } builder.append("\nChained subtasks: " + getChainedOutputs(cl)); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java index a300fc2585feb..74edb00823914 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamEdge.java @@ -28,6 +28,10 @@ */ public class StreamEdge implements Serializable { + private static final long serialVersionUID = 1L; + + final private String edgeId; + final private int sourceVertex; final private int targetVertex; @@ -48,6 +52,12 @@ public StreamEdge(int sourceVertex, int targetVertex, int typeNumber, List getPartitioner() { return outputPartitioner; } + @Override + public int hashCode() { + return edgeId.hashCode(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StreamEdge that = (StreamEdge) o; + + if (!edgeId.equals(that.edgeId)) { + return false; + } + + return true; + } + @Override public String toString() { - return "StreamGraphEdge{" + - "sourceVertex=" + sourceVertex + - ", targetVertex=" + targetVertex + + return "(" + + sourceVertex + + " -> " + targetVertex + ", typeNumber=" + typeNumber + ", selectedNames=" + selectedNames + ", outputPartitioner=" + outputPartitioner + - '}'; + ')'; } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java index d42003049bb63..8a110bfedf2fe 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamingJobGraphGenerator.java @@ -25,7 +25,6 @@ import java.util.Map; import org.apache.commons.lang.StringUtils; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.AbstractJobVertex; import org.apache.flink.runtime.jobgraph.DistributionPattern; @@ -53,7 +52,10 @@ public class StreamingJobGraphGenerator { private JobGraph jobGraph; private Collection builtVertices; + private List physicalEdgesInOrder; + private Map> chainedConfigs; + private Map vertexConfigs; private Map chainedNames; @@ -67,6 +69,7 @@ private void init() { this.chainedConfigs = new HashMap>(); this.vertexConfigs = new HashMap(); this.chainedNames = new HashMap(); + this.physicalEdgesInOrder = new ArrayList(); } public JobGraph createJobGraph(String jobName) { @@ -77,30 +80,57 @@ public JobGraph createJobGraph(String jobName) { jobGraph.setJobType(JobGraph.JobType.STREAMING); jobGraph.setMonitoringEnabled(streamGraph.isMonitoringEnabled()); jobGraph.setMonitorInterval(streamGraph.getMonitoringInterval()); - if(jobGraph.isMonitoringEnabled()) - { + if (jobGraph.isMonitoringEnabled()) { jobGraph.setNumberOfExecutionRetries(Integer.MAX_VALUE); } init(); setChaining(); + setPhysicalEdges(); + setSlotSharing(); return jobGraph; } + private void setPhysicalEdges() { + Map> physicalInEdgesInOrder = new HashMap>(); + + for (StreamEdge edge : physicalEdgesInOrder) { + int target = edge.getTargetVertex(); + + List inEdges = physicalInEdgesInOrder.get(target); + + // create if not set + if (inEdges == null) { + inEdges = new ArrayList(); + physicalInEdgesInOrder.put(target, inEdges); + } + + inEdges.add(edge); + } + + for (Map.Entry> inEdges : physicalInEdgesInOrder.entrySet()) { + int vertex = inEdges.getKey(); + List edgeList = inEdges.getValue(); + + vertexConfigs.get(vertex).setInPhysicalEdges(edgeList); + } + } + private void setChaining() { for (Integer sourceName : streamGraph.getSources()) { createChain(sourceName, sourceName); } } - private List> createChain(Integer startNode, Integer current) { + private List createChain(Integer startNode, Integer current) { if (!builtVertices.contains(startNode)) { - List> transitiveOutEdges = new ArrayList>(); + List transitiveOutEdges = new ArrayList(); + List chainableOutputs = new ArrayList(); List nonChainableOutputs = new ArrayList(); @@ -117,7 +147,7 @@ private List> createChain(Integer startNode, Integer cu } for (StreamEdge nonChainable : nonChainableOutputs) { - transitiveOutEdges.add(new Tuple2(current, nonChainable.getTargetVertex())); + transitiveOutEdges.add(nonChainable); createChain(nonChainable.getTargetVertex(), nonChainable.getTargetVertex()); } @@ -133,9 +163,8 @@ private List> createChain(Integer startNode, Integer cu config.setChainStart(); config.setOutEdgesInOrder(transitiveOutEdges); config.setOutEdges(streamGraph.getOutEdges(current)); - config.setInEdges(streamGraph.getInEdges(current)); - for (Tuple2 edge : transitiveOutEdges) { + for (StreamEdge edge : transitiveOutEdges) { connect(startNode, edge); } @@ -154,7 +183,7 @@ private List> createChain(Integer startNode, Integer cu return transitiveOutEdges; } else { - return new ArrayList>(); + return new ArrayList(); } } @@ -165,9 +194,11 @@ private String createChainedName(Integer vertexID, List chainedOutpu for (StreamEdge chainable : chainedOutputs) { outputChainedNames.add(chainedNames.get(chainable.getTargetVertex())); } - return operatorName + " -> (" + StringUtils.join(outputChainedNames, ", ") + ")"; + String returnOperatorName = operatorName + " -> (" + StringUtils.join(outputChainedNames, ", ") + ")"; + return returnOperatorName; } else if (chainedOutputs.size() == 1) { - return operatorName + " -> " + chainedNames.get(chainedOutputs.get(0)); + String returnOperatorName = operatorName + " -> " + chainedNames.get(chainedOutputs.get(0).getTargetVertex()); + return returnOperatorName; } else { return operatorName; } @@ -238,30 +269,20 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, vertexConfigs.put(vertexID, config); } - private void connect(Integer headOfChain, Tuple2 edge) { + private void connect(Integer headOfChain, StreamEdge edge) { - Integer upStreamvertexID = edge.f0; - Integer downStreamvertexID = edge.f1; + physicalEdgesInOrder.add(edge); - int outputIndex = streamGraph.getOutEdges(upStreamvertexID).indexOf(downStreamvertexID); + Integer downStreamvertexID = edge.getTargetVertex(); AbstractJobVertex headVertex = streamVertices.get(headOfChain); AbstractJobVertex downStreamVertex = streamVertices.get(downStreamvertexID); StreamConfig downStreamConfig = new StreamConfig(downStreamVertex.getConfiguration()); - StreamConfig upStreamConfig = headOfChain.equals(upStreamvertexID) ? new StreamConfig( - headVertex.getConfiguration()) : chainedConfigs.get(headOfChain).get( - upStreamvertexID); - - int numOfInputs = downStreamConfig.getNumberOfInputs(); - - downStreamConfig.setInputIndex(numOfInputs++, streamGraph.getEdge(upStreamvertexID, downStreamvertexID).getTypeNumber()); - downStreamConfig.setNumberOfInputs(numOfInputs); - - StreamPartitioner partitioner = streamGraph.getEdge(upStreamvertexID, downStreamvertexID).getPartitioner(); - upStreamConfig.setPartitioner(downStreamvertexID, partitioner); + downStreamConfig.setNumberOfInputs(downStreamConfig.getNumberOfInputs() + 1); + StreamPartitioner partitioner = edge.getPartitioner(); if (partitioner.getStrategy() == PartitioningStrategy.FORWARD) { downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE); } else { @@ -281,14 +302,15 @@ private boolean isChainable(StreamEdge edge) { StreamInvokable headInvokable = streamGraph.getInvokable(vertexID); StreamInvokable outInvokable = streamGraph.getInvokable(outName); - return streamGraph.getInEdges(outName).size() == 1 - && outInvokable != null - && outInvokable.getChainingStrategy() == ChainingStrategy.ALWAYS - && (headInvokable.getChainingStrategy() == ChainingStrategy.HEAD || headInvokable + return + streamGraph.getInEdges(outName).size() == 1 + && outInvokable != null + && outInvokable.getChainingStrategy() == ChainingStrategy.ALWAYS + && (headInvokable.getChainingStrategy() == ChainingStrategy.HEAD || headInvokable .getChainingStrategy() == ChainingStrategy.ALWAYS) - && streamGraph.getEdge(vertexID, outName).getPartitioner().getStrategy() == PartitioningStrategy.FORWARD - && streamGraph.getParallelism(vertexID) == streamGraph.getParallelism(outName) - && streamGraph.chaining; + && edge.getPartitioner().getStrategy() == PartitioningStrategy.FORWARD + && streamGraph.getParallelism(vertexID) == streamGraph.getParallelism(outName) + && streamGraph.chaining; } private void setSlotSharing() { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java index bb0268a49c3b6..4a0369c8fd370 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/CollectorWrapper.java @@ -29,7 +29,6 @@ public CollectorWrapper(OutputSelectorWrapper outputSelectorWrapper) { this.outputSelectorWrapper = outputSelectorWrapper; } - @SuppressWarnings("unchecked") public void addCollector(Collector output, StreamEdge edge) { outputSelectorWrapper.addCollector(output, edge); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java index 44371f096496d..78ef9141c768e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java @@ -25,12 +25,14 @@ public class BroadcastOutputSelectorWrapper implements OutputSelectorWrapper { + private static final long serialVersionUID = 1L; private List> outputs; public BroadcastOutputSelectorWrapper() { outputs = new ArrayList>(); } + @SuppressWarnings("unchecked") @Override public void addCollector(Collector output, StreamEdge edge) { outputs.add((Collector) output); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java index a04900c2eab11..1cb20d908ca05 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java @@ -31,6 +31,8 @@ public class DirectedOutputSelectorWrapper implements OutputSelectorWrapper { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DirectedOutputSelectorWrapper.class); private List> outputSelectors; @@ -44,6 +46,7 @@ public DirectedOutputSelectorWrapper(List> outputSelectors) this.outputMap = new HashMap>>(); } + @SuppressWarnings("unchecked") @Override public void addCollector(Collector output, StreamEdge edge) { List selectedNames = edge.getSelectedNames(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java index c0f22c7a0cfc3..dca2ede18535c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java @@ -21,6 +21,7 @@ public class OutputSelectorWrapperFactory { + @SuppressWarnings({ "rawtypes", "unchecked" }) public static OutputSelectorWrapper create(List> outputSelectors) { if (outputSelectors.size() == 0) { return new BroadcastOutputSelectorWrapper(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java index b45fc4436ab76..f277be052530f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java @@ -80,7 +80,7 @@ protected void setConfigInputs() throws StreamVertexException { ArrayList inputList1 = new ArrayList(); ArrayList inputList2 = new ArrayList(); - List inEdges = configuration.getInEdges(userClassLoader); + List inEdges = configuration.getInPhysicalEdges(userClassLoader); for (int i = 0; i < numberOfInputs; i++) { int inputType = inEdges.get(i).getTypeNumber(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java index 3988ec18f5bb2..c6a43773c444c 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java @@ -66,12 +66,6 @@ protected void setConfigInputs() throws StreamVertexException { } } - private IndexedReaderIterator> createInputIterator() { - final IndexedReaderIterator> iter = new IndexedReaderIterator>( - inputs, inputSerializer); - return iter; - } - protected static IndexedReaderIterator> staticCreateInputIterator( MutableReader inputReader, TypeSerializer> serializer) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java index bb66c63ffa8bb..42afcaec235f9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.plugable.SerializationDelegate; @@ -52,9 +51,10 @@ public class OutputHandler { public List> chainedInvokables; - private Map> outputMap; + private Map> outputMap; + private Map chainedConfigs; - private List> outEdgesInOrder; + private List outEdgesInOrder; public OutputHandler(StreamVertex vertex) { @@ -62,7 +62,7 @@ public OutputHandler(StreamVertex vertex) { this.vertex = vertex; this.configuration = new StreamConfig(vertex.getTaskConfiguration()); this.chainedInvokables = new ArrayList>(); - this.outputMap = new HashMap>(); + this.outputMap = new HashMap>(); this.cl = vertex.getUserCodeClassLoader(); // We read the chained configs, and the order of record writer @@ -74,16 +74,18 @@ public OutputHandler(StreamVertex vertex) { // We iterate through all the out edges from this job vertex and create // a stream output - for (Tuple2 outEdge : outEdgesInOrder) { - StreamOutput streamOutput = createStreamOutput(outEdge.f1, - chainedConfigs.get(outEdge.f0), outEdgesInOrder.indexOf(outEdge)); - outputMap.put(outEdge.f1, streamOutput); + for (StreamEdge outEdge : outEdgesInOrder) { + StreamOutput streamOutput = createStreamOutput( + outEdge, + outEdge.getTargetVertex(), + chainedConfigs.get(outEdge.getSourceVertex()), + outEdgesInOrder.indexOf(outEdge)); + outputMap.put(outEdge, streamOutput); } // We create the outer collector that will be passed to the first task // in the chain this.outerCollector = createChainedCollector(configuration); - } public void broadcastBarrier(long id) throws IOException, InterruptedException { @@ -120,9 +122,7 @@ private Collector createChainedCollector(StreamConfig chainedTaskConfig) { // Create collectors for the network outputs for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(cl)) { - Integer output = outputEdge.getTargetVertex(); - - Collector outCollector = outputMap.get(output); + Collector outCollector = outputMap.get(outputEdge); wrapper.addCollector(outCollector, outputEdge); } @@ -164,14 +164,14 @@ public Collector getCollector() { * * @param outputVertex * Name of the output to which the streamoutput will be set up - * @param configuration + * @param upStreamConfig * The config of upStream task - * @return + * @return The created StreamOutput */ - private StreamOutput createStreamOutput(Integer outputVertex, - StreamConfig configuration, int outputIndex) { + private StreamOutput createStreamOutput(StreamEdge edge, Integer outputVertex, + StreamConfig upStreamConfig, int outputIndex) { - StreamRecordSerializer outSerializer = configuration + StreamRecordSerializer outSerializer = upStreamConfig .getTypeSerializerOut1(vertex.userClassLoader); SerializationDelegate> outSerializationDelegate = null; @@ -180,12 +180,13 @@ private StreamOutput createStreamOutput(Integer outputVertex, outSerializationDelegate.setInstance(outSerializer.createInstance()); } - StreamPartitioner outputPartitioner = configuration.getPartitioner(cl, outputVertex); + @SuppressWarnings("unchecked") + StreamPartitioner outputPartitioner = (StreamPartitioner) edge.getPartitioner(); ResultPartitionWriter bufferWriter = vertex.getEnvironment().getWriter(outputIndex); RecordWriter>> output = - RecordWriterFactory.createRecordWriter(bufferWriter, outputPartitioner, configuration.getBufferTimeout()); + RecordWriterFactory.createRecordWriter(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout()); StreamOutput streamOutput = new StreamOutput(output, vertex.instanceID, outSerializationDelegate); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java index 20cd18914141d..f527de4f7207d 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java @@ -21,67 +21,101 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.function.co.CoMapFunction; +import org.apache.flink.streaming.api.function.co.CoFlatMapFunction; +import org.apache.flink.streaming.api.invokable.StreamInvokable; import org.apache.flink.streaming.util.TestListResultSink; import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.util.Collector; +import org.junit.Test; public class CoStreamTest { private static final long MEMORY_SIZE = 32; - private static ArrayList expected; + private static ArrayList expected = new ArrayList(); - public static void main(String[] args) throws InterruptedException { - for (int i = 0; i < 200; i++) { - test(); - } - } + @Test + public void test() { - // @Test - public static void test() { - expected = new ArrayList(); - - StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE); + StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE); TestListResultSink resultSink = new TestListResultSink(); DataStream src = env.fromElements(1, 3, 5); - DataStream src2 = env.fromElements(1, 3, 5); - DataStream grouped = src.groupBy(new KeySelector() { + DataStream filter1 = src.filter(new FilterFunction() { + + private static final long serialVersionUID = 1L; + @Override - public Integer getKey(Integer value) throws Exception { - return value; + public boolean filter(Integer value) throws Exception { + return true; } - }); + }).groupBy(new KeySelector() { + + private static final long serialVersionUID = 1L; - DataStream grouped2 = src2.groupBy(new KeySelector() { @Override public Integer getKey(Integer value) throws Exception { return value; } }); - DataStream connected = grouped.connect(grouped2).map(new CoMapFunction() { + DataStream> filter2 = src + .map(new MapFunction>() { + + private static final long serialVersionUID = 1L; + + @Override + public Tuple2 map(Integer value) throws Exception { + return new Tuple2(value, value + 1); + } + }) + .distribute() + .filter(new FilterFunction>() { + + private static final long serialVersionUID = 1L; + + @Override + public boolean filter(Tuple2 value) throws Exception { + return true; + } + }).setChainingStrategy(StreamInvokable.ChainingStrategy.NEVER).groupBy(new KeySelector, Integer>() { + + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(Tuple2 value) throws Exception { + return value.f0; + } + }); + + DataStream connected = filter1.connect(filter2).flatMap(new CoFlatMapFunction, String>() { + + private static final long serialVersionUID = 1L; + @Override - public String map1(Integer value) { - return value.toString(); + public void flatMap1(Integer value, Collector out) throws Exception { + out.collect(value.toString()); } @Override - public String map2(Integer value) { - return value.toString(); + public void flatMap2(Tuple2 value, Collector out) throws Exception { + out.collect(value.toString()); } }); connected.addSink(resultSink); - connected.print(); - try { env.execute(); } catch (Exception e) { @@ -89,9 +123,11 @@ public String map2(Integer value) { } expected = new ArrayList(); - expected.addAll(Arrays.asList("1", "1", "3", "3", "5", "5")); + expected.addAll(Arrays.asList("(1,2)", "(3,4)", "(5,6)", "1", "3", "5")); + + List result = resultSink.getResult(); + Collections.sort(result); - System.out.println(resultSink.getResult()); - assertEquals(expected, expected); + assertEquals(expected, result); } } \ No newline at end of file diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/co/SelfConnectionTest.java new file mode 100644 index 0000000000000..b58245a9724d0 --- /dev/null +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/co/SelfConnectionTest.java @@ -0,0 +1,252 @@ +/* + * 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.invokable.operator.co; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.function.co.CoMapFunction; +import org.apache.flink.streaming.api.invokable.StreamInvokable; +import org.apache.flink.streaming.api.windowing.helper.Timestamp; +import org.apache.flink.streaming.util.TestListResultSink; +import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.util.Collector; +import org.junit.Test; + +public class SelfConnectionTest implements Serializable { + + private static final long serialVersionUID = 1L; + + private final int MEMORY_SIZE = 32; + + private static List expected; + + @SuppressWarnings({ "unchecked", "rawtypes" }) + @Test + public void sameDataStreamTest() { + + StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE); + + TestListResultSink resultSink = new TestListResultSink(); + + Timestamp timeStamp = new Timestamp() { + + private static final long serialVersionUID = 1L; + + @Override + public long getTimestamp(Integer value) { + return value; + } + + }; + + KeySelector keySelector = new KeySelector() { + + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + }; + + DataStream src = env.fromElements(1, 3, 5); + + @SuppressWarnings("unused") + DataStream> dataStream = + src.join(src).onWindow(50L, timeStamp, timeStamp).where(keySelector).equalTo(keySelector) + .map(new MapFunction, String>() { + + private static final long serialVersionUID = 1L; + + @Override + public String map(Tuple2 value) throws Exception { + return value.toString(); + } + }) + .addSink(resultSink); + + + try { + env.execute(); + + expected = new ArrayList(); + + expected.addAll(Arrays.asList("(1,1)", "(3,3)", "(5,5)")); + + List result = resultSink.getResult(); + + Collections.sort(expected); + Collections.sort(result); + + assertEquals(expected, result); + } catch (Exception e) { + fail(); + e.printStackTrace(); + } + } + + /** + * We connect two different data streams in a chain to a CoMap. + */ + @Test + public void differentDataStreamSameChain() { + + TestListResultSink resultSink = new TestListResultSink(); + + StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE); + + DataStream src = env.fromElements(1, 3, 5); + + DataStream stringMap = src.map(new MapFunction() { + private static final long serialVersionUID = 1L; + + @Override + public String map(Integer value) throws Exception { + return "x " + value; + } + }).setChainingStrategy(StreamInvokable.ChainingStrategy.ALWAYS); + + stringMap.connect(src).map(new CoMapFunction() { + + private static final long serialVersionUID = 1L; + + @Override + public String map1(String value) { + return value; + } + + @Override + public String map2(Integer value) { + return String.valueOf(value + 1); + } + }).addSink(resultSink); + + try { + env.execute(); + } catch (Exception e) { + e.printStackTrace(); + } + + expected = new ArrayList(); + + expected.addAll(Arrays.asList("x 1", "x 3", "x 5", "2", "4", "6")); + + List result = resultSink.getResult(); + + Collections.sort(expected); + Collections.sort(result); + + assertEquals(expected, result); + } + + /** + * We connect two different data streams in different chains to a CoMap. + * (This is not actually self-connect.) + */ + @Test + public void differentDataStreamDifferentChain() { + + TestListResultSink resultSink = new TestListResultSink(); + + StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE); + + DataStream src = env.fromElements(1, 3, 5).setChainingStrategy(StreamInvokable.ChainingStrategy.NEVER); + + DataStream stringMap = src.flatMap(new FlatMapFunction() { + + private static final long serialVersionUID = 1L; + + @Override + public void flatMap(Integer value, Collector out) throws Exception { + out.collect("x " + value); + } + }).groupBy(new KeySelector() { + + private static final long serialVersionUID = 1L; + + @Override + public Integer getKey(String value) throws Exception { + return value.length(); + } + }); + + DataStream longMap = src.map(new MapFunction() { + + private static final long serialVersionUID = 1L; + + @Override + public Long map(Integer value) throws Exception { + return Long.valueOf(value + 1); + } + }).groupBy(new KeySelector() { + + private static final long serialVersionUID = 1L; + + @Override + public Long getKey(Long value) throws Exception { + return value; + } + }); + + + stringMap.connect(longMap).map(new CoMapFunction() { + + private static final long serialVersionUID = 1L; + + @Override + public String map1(String value) { + return value; + } + + @Override + public String map2(Long value) { + return value.toString(); + } + }).addSink(resultSink); + + try { + env.execute(); + } catch (Exception e) { + e.printStackTrace(); + } + + expected = new ArrayList(); + + expected.addAll(Arrays.asList("x 1", "x 3", "x 5", "2", "4", "6")); + + List result = resultSink.getResult(); + + Collections.sort(expected); + Collections.sort(result); + + assertEquals(expected, result); + } +} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java index 18a36ac525394..a88a60d5193ae 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java @@ -134,11 +134,13 @@ private static class CoMap implements CoMapFunction { @Override public String map1(String value) { +// System.out.println(value); return value; } @Override public String map2(Long value) { +// System.out.println(value); return value.toString(); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java index 7f72173cdf17e..8b78a42d7ab37 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java @@ -27,6 +27,7 @@ public class TestListResultSink extends RichSinkFunction { + private static final long serialVersionUID = 1L; private int resultListId; public TestListResultSink() { @@ -50,6 +51,7 @@ public void close() throws Exception { super.close(); } + @SuppressWarnings("unchecked") private List resultList() { synchronized (TestListWrapper.getInstance()) { return (List) TestListWrapper.getInstance().getList(resultListId); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java index 3c50f63009f0f..751f8360f13e6 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java @@ -25,8 +25,10 @@ public class TestListWrapper { private static TestListWrapper instance; + @SuppressWarnings("rawtypes") private List> lists; + @SuppressWarnings("rawtypes") private TestListWrapper() { lists = Collections.synchronizedList(new ArrayList>()); } @@ -43,12 +45,14 @@ public static TestListWrapper getInstance() { * * @return The ID of the list. */ + @SuppressWarnings("rawtypes") public int createList() { lists.add(new ArrayList()); return lists.size() - 1; } public List getList(int listId) { + @SuppressWarnings("rawtypes") List list = lists.get(listId); if (list == null) { throw new RuntimeException("No such list.");