提交 f0cc5d64 编写于 作者: G Gyula Fora 提交者: mbalassi

[FLINK-1279] [streaming] Forward partitioning changed to use round-robin method

上级 64baa00b
...@@ -37,8 +37,8 @@ import org.apache.flink.streaming.api.streamvertex.CoStreamVertex; ...@@ -37,8 +37,8 @@ import org.apache.flink.streaming.api.streamvertex.CoStreamVertex;
import org.apache.flink.streaming.api.streamvertex.StreamIterationHead; import org.apache.flink.streaming.api.streamvertex.StreamIterationHead;
import org.apache.flink.streaming.api.streamvertex.StreamIterationTail; import org.apache.flink.streaming.api.streamvertex.StreamIterationTail;
import org.apache.flink.streaming.api.streamvertex.StreamVertex; import org.apache.flink.streaming.api.streamvertex.StreamVertex;
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner; import org.apache.flink.streaming.partitioner.StreamPartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner.PartitioningStrategy;
import org.apache.flink.streaming.state.OperatorState; import org.apache.flink.streaming.state.OperatorState;
import org.apache.flink.streaming.util.serialization.TypeWrapper; import org.apache.flink.streaming.util.serialization.TypeWrapper;
import org.slf4j.Logger; import org.slf4j.Logger;
...@@ -80,7 +80,6 @@ public class JobGraphBuilder { ...@@ -80,7 +80,6 @@ public class JobGraphBuilder {
private Map<String, Long> iterationWaitTime; private Map<String, Long> iterationWaitTime;
private Map<String, Map<String, OperatorState<?>>> operatorStates; private Map<String, Map<String, OperatorState<?>>> operatorStates;
/** /**
* Creates an new {@link JobGraph} with the given name. A JobGraph is a DAG * Creates an new {@link JobGraph} with the given name. A JobGraph is a DAG
* and consists of sources, tasks (intermediate vertices) and sinks. * and consists of sources, tasks (intermediate vertices) and sinks.
...@@ -205,7 +204,7 @@ public class JobGraphBuilder { ...@@ -205,7 +204,7 @@ public class JobGraphBuilder {
*/ */
public void addIterationTail(String vertexName, String iterationTail, String iterationID, public void addIterationTail(String vertexName, String iterationTail, String iterationID,
int parallelism, long waitTime) { int parallelism, long waitTime) {
if (bufferTimeout.get(iterationTail) == 0) { if (bufferTimeout.get(iterationTail) == 0) {
throw new RuntimeException("Buffer timeout 0 at iteration tail is not supported."); throw new RuntimeException("Buffer timeout 0 at iteration tail is not supported.");
} }
...@@ -355,7 +354,7 @@ public class JobGraphBuilder { ...@@ -355,7 +354,7 @@ public class JobGraphBuilder {
StreamConfig config = new StreamConfig(upStreamVertex.getConfiguration()); StreamConfig config = new StreamConfig(upStreamVertex.getConfiguration());
if (partitionerObject.getClass().equals(ForwardPartitioner.class)) { if (partitionerObject.getStrategy() == PartitioningStrategy.FORWARD) {
downStreamVertex downStreamVertex
.connectNewDataSetAsInput(upStreamVertex, DistributionPattern.POINTWISE); .connectNewDataSetAsInput(upStreamVertex, DistributionPattern.POINTWISE);
} else { } else {
......
...@@ -66,7 +66,6 @@ import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy; ...@@ -66,7 +66,6 @@ import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
import org.apache.flink.streaming.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
import org.apache.flink.streaming.partitioner.DistributePartitioner; import org.apache.flink.streaming.partitioner.DistributePartitioner;
import org.apache.flink.streaming.partitioner.FieldsPartitioner; import org.apache.flink.streaming.partitioner.FieldsPartitioner;
import org.apache.flink.streaming.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.partitioner.ShufflePartitioner; import org.apache.flink.streaming.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.partitioner.StreamPartitioner; import org.apache.flink.streaming.partitioner.StreamPartitioner;
import org.apache.flink.streaming.util.keys.FieldsKeySelector; import org.apache.flink.streaming.util.keys.FieldsKeySelector;
...@@ -127,7 +126,7 @@ public class DataStream<OUT> { ...@@ -127,7 +126,7 @@ public class DataStream<OUT> {
this.jobGraphBuilder = environment.getJobGraphBuilder(); this.jobGraphBuilder = environment.getJobGraphBuilder();
this.userDefinedNames = new ArrayList<String>(); this.userDefinedNames = new ArrayList<String>();
this.selectAll = false; this.selectAll = false;
this.partitioner = new ForwardPartitioner<OUT>(); this.partitioner = new DistributePartitioner<OUT>(true);
this.outTypeWrapper = outTypeWrapper; this.outTypeWrapper = outTypeWrapper;
this.mergedStreams = new ArrayList<DataStream<OUT>>(); this.mergedStreams = new ArrayList<DataStream<OUT>>();
this.mergedStreams.add(this); this.mergedStreams.add(this);
...@@ -158,13 +157,6 @@ public class DataStream<OUT> { ...@@ -158,13 +157,6 @@ public class DataStream<OUT> {
} }
/**
* Partitioning strategy on the stream.
*/
public static enum ConnectionType {
SHUFFLE, BROADCAST, FIELD, FORWARD, DISTRIBUTE
}
/** /**
* Returns the ID of the {@link DataStream}. * Returns the ID of the {@link DataStream}.
* *
...@@ -341,7 +333,7 @@ public class DataStream<OUT> { ...@@ -341,7 +333,7 @@ public class DataStream<OUT> {
* @return The DataStream with shuffle partitioning set. * @return The DataStream with shuffle partitioning set.
*/ */
public DataStream<OUT> forward() { public DataStream<OUT> forward() {
return setConnectionType(new ForwardPartitioner<OUT>()); return setConnectionType(new DistributePartitioner<OUT>(true));
} }
/** /**
...@@ -351,7 +343,7 @@ public class DataStream<OUT> { ...@@ -351,7 +343,7 @@ public class DataStream<OUT> {
* @return The DataStream with shuffle partitioning set. * @return The DataStream with shuffle partitioning set.
*/ */
public DataStream<OUT> distribute() { public DataStream<OUT> distribute() {
return setConnectionType(new DistributePartitioner<OUT>()); return setConnectionType(new DistributePartitioner<OUT>(false));
} }
/** /**
......
...@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.datastream; ...@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.datastream;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import org.apache.flink.streaming.partitioner.ForwardPartitioner; import org.apache.flink.streaming.partitioner.DistributePartitioner;
/** /**
* The iterative data stream represents the start of an iteration in a * The iterative data stream represents the start of an iteration in a
...@@ -91,8 +91,8 @@ public class IterativeDataStream<IN> extends ...@@ -91,8 +91,8 @@ public class IterativeDataStream<IN> extends
for (DataStream<IN> stream : iterationTail.mergedStreams) { for (DataStream<IN> stream : iterationTail.mergedStreams) {
String inputID = stream.getId(); String inputID = stream.getId();
jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<IN>(), 0, jobGraphBuilder.setEdge(inputID, returnStream.getId(), new DistributePartitioner<IN>(
name, false); true), 0, name, false);
} }
return iterationTail; return iterationTail;
......
...@@ -26,16 +26,21 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord; ...@@ -26,16 +26,21 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T> * @param <T>
* Type of the Tuple * Type of the Tuple
*/ */
public class BroadcastPartitioner<T> implements StreamPartitioner<T> { public class BroadcastPartitioner<T> extends StreamPartitioner<T> {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
int[] returnArray; int[] returnArray;
boolean set; boolean set;
int setNumber;
public BroadcastPartitioner() {
super(PartitioningStrategy.BROADCAST);
}
@Override @Override
public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record, public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
int numberOfOutputChannels) { int numberOfOutputChannels) {
if (set) { if (set && setNumber == numberOfOutputChannels) {
return returnArray; return returnArray;
} else { } else {
this.returnArray = new int[numberOfOutputChannels]; this.returnArray = new int[numberOfOutputChannels];
...@@ -43,6 +48,7 @@ public class BroadcastPartitioner<T> implements StreamPartitioner<T> { ...@@ -43,6 +48,7 @@ public class BroadcastPartitioner<T> implements StreamPartitioner<T> {
returnArray[i] = i; returnArray[i] = i;
} }
set = true; set = true;
setNumber = numberOfOutputChannels;
return returnArray; return returnArray;
} }
} }
......
...@@ -27,22 +27,20 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord; ...@@ -27,22 +27,20 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T> * @param <T>
* Type of the Tuple * Type of the Tuple
*/ */
public class DistributePartitioner<T> implements StreamPartitioner<T> { public class DistributePartitioner<T> extends StreamPartitioner<T> {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
private int currentChannelIndex; private int[] returnArray = new int[] {-1};
private int[] returnArray;
public DistributePartitioner(boolean forward) {
public DistributePartitioner() { super(forward ? PartitioningStrategy.FORWARD : PartitioningStrategy.DISTRIBUTE);
this.currentChannelIndex = 0;
this.returnArray = new int[1];
} }
@Override @Override
public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record, public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
int numberOfOutputChannels) { int numberOfOutputChannels) {
returnArray[0] = currentChannelIndex; this.returnArray[0] = (this.returnArray[0] + 1) % numberOfOutputChannels;
currentChannelIndex = (currentChannelIndex + 1) % numberOfOutputChannels;
return returnArray; return this.returnArray;
} }
} }
...@@ -28,15 +28,15 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord; ...@@ -28,15 +28,15 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T> * @param <T>
* Type of the Tuple * Type of the Tuple
*/ */
public class FieldsPartitioner<T> implements StreamPartitioner<T> { public class FieldsPartitioner<T> extends StreamPartitioner<T> {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
private int[] returnArray; private int[] returnArray = new int[1];;
KeySelector<T, ?> keySelector; KeySelector<T, ?> keySelector;
public FieldsPartitioner(KeySelector<T, ?> keySelector) { public FieldsPartitioner(KeySelector<T, ?> keySelector) {
super(PartitioningStrategy.FIELDS);
this.keySelector = keySelector; this.keySelector = keySelector;
this.returnArray = new int[1];
} }
@Override @Override
......
/*
* 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.partitioner;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/**
* Partitioner that forwards the tuples to the local subtask of the output vertex
*
* @param <T>
* Type of the Tuple
*/
public class ForwardPartitioner<T> implements StreamPartitioner<T> {
private static final long serialVersionUID = 1L;
private int[] returnArray;
public ForwardPartitioner() {
this.returnArray = new int[]{0};
}
@Override
public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
int numberOfOutputChannels) {
return returnArray;
}
}
...@@ -21,13 +21,13 @@ import org.apache.flink.runtime.plugable.SerializationDelegate; ...@@ -21,13 +21,13 @@ import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord; import org.apache.flink.streaming.api.streamrecord.StreamRecord;
//Group to the partitioner with the lowest id //Group to the partitioner with the lowest id
public class GlobalPartitioner<T> implements StreamPartitioner<T> { public class GlobalPartitioner<T> extends StreamPartitioner<T> {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
private int[] returnArray; private int[] returnArray = new int[] { 0 };
public GlobalPartitioner() { public GlobalPartitioner() {
this.returnArray = new int[] { 0 }; super(PartitioningStrategy.GLOBAL);
} }
@Override @Override
......
...@@ -29,16 +29,15 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord; ...@@ -29,16 +29,15 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
* @param <T> * @param <T>
* Type of the Tuple * Type of the Tuple
*/ */
public class ShufflePartitioner<T> implements StreamPartitioner<T> { public class ShufflePartitioner<T> extends StreamPartitioner<T> {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
private Random random = new Random(); private Random random = new Random();
private int[] returnArray; private int[] returnArray = new int[1];
public ShufflePartitioner() { public ShufflePartitioner() {
this.random = new Random(); super(PartitioningStrategy.SHUFFLE);
this.returnArray = new int[1];
} }
@Override @Override
......
...@@ -22,12 +22,23 @@ import org.apache.flink.runtime.io.network.api.ChannelSelector; ...@@ -22,12 +22,23 @@ import org.apache.flink.runtime.io.network.api.ChannelSelector;
import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.streamrecord.StreamRecord; import org.apache.flink.streaming.api.streamrecord.StreamRecord;
/** public abstract class StreamPartitioner<T> implements
* Empty interface to encapsulate partitioners. ChannelSelector<SerializationDelegate<StreamRecord<T>>>, Serializable {
*
* @param <T> public enum PartitioningStrategy {
* Type of the Tuple
*/ FORWARD, DISTRIBUTE, SHUFFLE, BROADCAST, GLOBAL, FIELDS;
public interface StreamPartitioner<T> extends ChannelSelector<SerializationDelegate<StreamRecord<T>>>,
Serializable { }
private static final long serialVersionUID = 1L;
private PartitioningStrategy strategy;
public StreamPartitioner(PartitioningStrategy strategy) {
this.strategy = strategy;
}
public PartitioningStrategy getStrategy() {
return strategy;
}
} }
...@@ -27,6 +27,7 @@ import org.junit.Test; ...@@ -27,6 +27,7 @@ import org.junit.Test;
public class PrintTest implements Serializable { public class PrintTest implements Serializable {
private static final long serialVersionUID = 1L;
private static final long MEMORYSIZE = 32; private static final long MEMORYSIZE = 32;
private static final class IdentityMap implements MapFunction<Long, Long> { private static final class IdentityMap implements MapFunction<Long, Long> {
......
...@@ -34,7 +34,7 @@ public class DistributePartitionerTest { ...@@ -34,7 +34,7 @@ public class DistributePartitionerTest {
@Before @Before
public void setPartitioner() { public void setPartitioner() {
distributePartitioner = new DistributePartitioner<Tuple>(); distributePartitioner = new DistributePartitioner<Tuple>(false);
} }
@Test @Test
......
...@@ -26,17 +26,17 @@ import org.junit.Before; ...@@ -26,17 +26,17 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
public class ForwardPartitionerTest { public class ForwardPartitionerTest {
private ForwardPartitioner<Tuple> forwardPartitioner; private DistributePartitioner<Tuple> forwardPartitioner;
private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(); private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>();
private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>( private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
null); null);
@Before @Before
public void setPartitioner() { public void setPartitioner() {
forwardPartitioner = new ForwardPartitioner<Tuple>(); forwardPartitioner = new DistributePartitioner<Tuple>(true);
} }
@Test @Test
public void testSelectChannelsLength() { public void testSelectChannelsLength() {
sd.setInstance(streamRecord); sd.setInstance(streamRecord);
...@@ -44,12 +44,12 @@ public class ForwardPartitionerTest { ...@@ -44,12 +44,12 @@ public class ForwardPartitionerTest {
assertEquals(1, forwardPartitioner.selectChannels(sd, 2).length); assertEquals(1, forwardPartitioner.selectChannels(sd, 2).length);
assertEquals(1, forwardPartitioner.selectChannels(sd, 1024).length); assertEquals(1, forwardPartitioner.selectChannels(sd, 1024).length);
} }
@Test @Test
public void testSelectChannelsInterval() { public void testSelectChannelsInterval() {
sd.setInstance(streamRecord); sd.setInstance(streamRecord);
assertEquals(0, forwardPartitioner.selectChannels(sd, 1)[0]); assertEquals(0, forwardPartitioner.selectChannels(sd, 1)[0]);
assertEquals(0, forwardPartitioner.selectChannels(sd, 2)[0]); assertEquals(1, forwardPartitioner.selectChannels(sd, 2)[0]);
assertEquals(0, forwardPartitioner.selectChannels(sd, 1024)[0]); assertEquals(2, forwardPartitioner.selectChannels(sd, 1024)[0]);
} }
} }
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册