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

[FLINK-1380] [streaming] Updated SplitDataStream to extend DataStream to get...

[FLINK-1380] [streaming] Updated SplitDataStream to extend DataStream to get rid of selectAll method for splits
上级 d908ca19
...@@ -475,7 +475,7 @@ DataStream<Integer> odd = split.select("odd"); ...@@ -475,7 +475,7 @@ DataStream<Integer> odd = split.select("odd");
In the above example the data stream named ‘even’ will only contain elements that are directed to the output named “even”. The user can of course further transform these new stream by for example squaring only the even elements. In the above example the data stream named ‘even’ will only contain elements that are directed to the output named “even”. The user can of course further transform these new stream by for example squaring only the even elements.
Data streams only receive the elements directed to selected output names. The user can also select multiple output names by `splitStream.select(“output1”, “output2”…)`. It is common that a stream listens to all the outputs, so `split.selectAll()` provides this functionality without having to select all names. Data streams only receive the elements directed to selected output names. The user can also select multiple output names by `splitStream.select(“output1”, “output2”…)`. It is common that a stream listens to all the outputs, by simply applying the transformation on the split data stream without select provides this functionality.
The outputs of an operator are directed by implementing a selector function (implementing the `OutputSelector` interface): The outputs of an operator are directed by implementing a selector function (implementing the `OutputSelector` interface):
......
...@@ -124,7 +124,7 @@ public class DataStream<OUT> { ...@@ -124,7 +124,7 @@ public class DataStream<OUT> {
this.degreeOfParallelism = environment.getDegreeOfParallelism(); this.degreeOfParallelism = environment.getDegreeOfParallelism();
this.jobGraphBuilder = environment.getJobGraphBuilder(); this.jobGraphBuilder = environment.getJobGraphBuilder();
this.userDefinedNames = new ArrayList<String>(); this.userDefinedNames = new ArrayList<String>();
this.selectAll = false; this.selectAll = true;
this.partitioner = new DistributePartitioner<OUT>(true); this.partitioner = new DistributePartitioner<OUT>(true);
this.typeInfo = typeInfo; this.typeInfo = typeInfo;
this.mergedStreams = new ArrayList<DataStream<OUT>>(); this.mergedStreams = new ArrayList<DataStream<OUT>>();
......
...@@ -41,14 +41,21 @@ import org.apache.flink.streaming.state.OperatorState; ...@@ -41,14 +41,21 @@ import org.apache.flink.streaming.state.OperatorState;
public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperator<OUT, O>> extends public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperator<OUT, O>> extends
DataStream<OUT> { DataStream<OUT> {
protected boolean isSplit;
protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, protected SingleOutputStreamOperator(StreamExecutionEnvironment environment,
String operatorType, TypeInformation<OUT> outTypeInfo) { String operatorType, TypeInformation<OUT> outTypeInfo) {
super(environment, operatorType, outTypeInfo); super(environment, operatorType, outTypeInfo);
setBufferTimeout(environment.getBufferTimeout()); setBufferTimeout(environment.getBufferTimeout());
this.isSplit = false;
} }
@SuppressWarnings("unchecked")
protected SingleOutputStreamOperator(DataStream<OUT> dataStream) { protected SingleOutputStreamOperator(DataStream<OUT> dataStream) {
super(dataStream); super(dataStream);
if (dataStream instanceof SingleOutputStreamOperator) {
this.isSplit = ((SingleOutputStreamOperator<OUT, ?>) dataStream).isSplit;
}
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
...@@ -101,15 +108,21 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato ...@@ -101,15 +108,21 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
* @return The {@link SplitDataStream} * @return The {@link SplitDataStream}
*/ */
public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) { public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) {
try { if (!isSplit) {
jobGraphBuilder.setOutputSelector(id, this.isSplit = true;
SerializationUtils.serialize(clean(outputSelector))); try {
jobGraphBuilder.setOutputSelector(id,
} catch (SerializationException e) { SerializationUtils.serialize(clean(outputSelector)));
throw new RuntimeException("Cannot serialize OutputSelector");
} catch (SerializationException e) {
throw new RuntimeException("Cannot serialize OutputSelector");
}
return new SplitDataStream<OUT>(this);
} else {
throw new RuntimeException("Currently operators can only be split once");
} }
return new SplitDataStream<OUT>(this);
} }
/** /**
......
/* /*
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership. * this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0 * 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 not use this file except in compliance with
* the License. You may obtain a copy of the License at * the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.flink.streaming.api.datastream;
import java.util.Arrays;
import org.apache.flink.api.common.typeinfo.TypeInformation; package org.apache.flink.streaming.api.datastream;
import org.apache.flink.streaming.api.collector.OutputSelector;
import java.util.Arrays;
/**
* The SplitDataStream represents an operator that has been split using an import org.apache.flink.streaming.api.collector.OutputSelector;
* {@link OutputSelector}. Named outputs can be selected using the
* {@link #select} function. /**
* * The SplitDataStream represents an operator that has been split using an
* @param <OUT> * {@link OutputSelector}. Named outputs can be selected using the
* The type of the output. * {@link #select} function. To apply transformation on the whole output simply
*/ * call the transformation on the SplitDataStream
public class SplitDataStream<OUT> { *
* @param <OUT>
DataStream<OUT> dataStream; * The type of the output.
*/
protected SplitDataStream(DataStream<OUT> dataStream) { public class SplitDataStream<OUT> extends DataStream<OUT> {
this.dataStream = dataStream.copy();
} protected SplitDataStream(DataStream<OUT> dataStream) {
super(dataStream);
}
/** /**
* Gets the output type. * Sets the output names for which the next operator will receive values.
* *
* @return The output type. * @param outputNames
* The output names for which the operator will receive the
* input.
* @return Returns the selected DataStream
*/ */
public TypeInformation<OUT> getOutputType() { public DataStream<OUT> select(String... outputNames) {
return dataStream.getType(); return selectOutput(outputNames);
}
private DataStream<OUT> selectOutput(String[] outputNames) {
DataStream<OUT> returnStream = copy();
returnStream.selectAll = false;
returnStream.userDefinedNames = Arrays.asList(outputNames);
return returnStream;
} }
/** }
* Sets the output names for which the next operator will receive values.
*
* @param outputNames
* The output names for which the operator will receive the
* input.
* @return Returns the selected DataStream
*/
public DataStream<OUT> select(String... outputNames) {
return selectOutput(outputNames);
}
/**
* Selects all output names from a split data stream.
*
* @return Returns the selected DataStream
*/
public DataStream<OUT> selectAll() {
DataStream<OUT> returnStream = dataStream.copy();
returnStream.selectAll = true;
return returnStream;
}
private DataStream<OUT> selectOutput(String[] outputNames) {
DataStream<OUT> returnStream = dataStream.copy();
returnStream.userDefinedNames = Arrays.asList(outputNames);
return returnStream;
}
}
...@@ -100,7 +100,7 @@ public class DirectedOutputTest { ...@@ -100,7 +100,7 @@ public class DirectedOutputTest {
source.select(EVEN).addSink(new ListSink(EVEN)); source.select(EVEN).addSink(new ListSink(EVEN));
source.select(ODD, TEN).addSink(new ListSink(ODD_AND_TEN)); source.select(ODD, TEN).addSink(new ListSink(ODD_AND_TEN));
source.select(EVEN, ODD).addSink(new ListSink(EVEN_AND_ODD)); source.select(EVEN, ODD).addSink(new ListSink(EVEN_AND_ODD));
source.selectAll().addSink(new ListSink(ALL)); source.addSink(new ListSink(ALL));
env.execute(); env.execute();
assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), outputs.get(EVEN)); assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), outputs.get(EVEN));
......
...@@ -23,26 +23,17 @@ import org.apache.flink.streaming.api.datastream.{ SplitDataStream => SplitJavaS ...@@ -23,26 +23,17 @@ import org.apache.flink.streaming.api.datastream.{ SplitDataStream => SplitJavaS
/** /**
* The SplitDataStream represents an operator that has been split using an * The SplitDataStream represents an operator that has been split using an
* {@link OutputSelector}. Named outputs can be selected using the * {@link OutputSelector}. Named outputs can be selected using the
* {@link #select} function. * {@link #select} function. To apply a transformation on the whole output simply call
* the appropriate method on this stream.
* *
* @param <OUT> * @param <OUT>
* The type of the output. * The type of the output.
*/ */
class SplitDataStream[T](javaStream: SplitJavaStream[T]) { class SplitDataStream[T](javaStream: SplitJavaStream[T]) extends DataStream[T](javaStream){
/**
* Gets the underlying java DataStream object.
*/
private[flink] def getJavaStream: SplitJavaStream[T] = javaStream
/** /**
* Sets the output names for which the next operator will receive values. * Sets the output names for which the next operator will receive values.
*/ */
def select(outputNames: String*): DataStream[T] = javaStream.select(outputNames: _*) def select(outputNames: String*): DataStream[T] = javaStream.select(outputNames: _*)
/**
* Selects all output names from a split data stream.
*/
def selectAll(): DataStream[T] = javaStream.selectAll()
} }
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册