diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md index 7808b9e53060a7a7c678fbdb187fb892db16d2ea..1b0a1e9c0959f0a319060c7c6a4008588e01cdba 100644 --- a/docs/streaming_guide.md +++ b/docs/streaming_guide.md @@ -475,7 +475,7 @@ DataStream 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. -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): diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 8e2121813679195fc97b66200371435e6ac09926..7d1659f933f4114bce377ac28db6b6df8ea63b37 100644 --- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -124,7 +124,7 @@ public class DataStream { this.degreeOfParallelism = environment.getDegreeOfParallelism(); this.jobGraphBuilder = environment.getJobGraphBuilder(); this.userDefinedNames = new ArrayList(); - this.selectAll = false; + this.selectAll = true; this.partitioner = new DistributePartitioner(true); this.typeInfo = typeInfo; this.mergedStreams = new ArrayList>(); diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java index aa8557913a118cbb8e0fff1f66fb76acac466f31..5a8261ecb918d8789f453802a998d9a6f8f0fa8b 100755 --- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java +++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java @@ -41,14 +41,21 @@ import org.apache.flink.streaming.state.OperatorState; public class SingleOutputStreamOperator> extends DataStream { + protected boolean isSplit; + protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, String operatorType, TypeInformation outTypeInfo) { super(environment, operatorType, outTypeInfo); setBufferTimeout(environment.getBufferTimeout()); + this.isSplit = false; } + @SuppressWarnings("unchecked") protected SingleOutputStreamOperator(DataStream dataStream) { super(dataStream); + if (dataStream instanceof SingleOutputStreamOperator) { + this.isSplit = ((SingleOutputStreamOperator) dataStream).isSplit; + } } @SuppressWarnings("unchecked") @@ -101,15 +108,21 @@ public class SingleOutputStreamOperator split(OutputSelector outputSelector) { - try { - jobGraphBuilder.setOutputSelector(id, - SerializationUtils.serialize(clean(outputSelector))); - - } catch (SerializationException e) { - throw new RuntimeException("Cannot serialize OutputSelector"); + if (!isSplit) { + this.isSplit = true; + try { + jobGraphBuilder.setOutputSelector(id, + SerializationUtils.serialize(clean(outputSelector))); + + } catch (SerializationException e) { + throw new RuntimeException("Cannot serialize OutputSelector"); + } + + return new SplitDataStream(this); + } else { + throw new RuntimeException("Currently operators can only be split once"); } - return new SplitDataStream(this); } /** diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java index 4fac04c1a1da1de7375ea0ca6241aaed4d70f2cd..1bf4f9c63319ebe190a6d859697f9b7bbe8b0404 100755 --- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java +++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java @@ -1,79 +1,58 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and * limitations under the License. - */ - -package org.apache.flink.streaming.api.datastream; - -import java.util.Arrays; + */ -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.collector.OutputSelector; - -/** - * The SplitDataStream represents an operator that has been split using an - * {@link OutputSelector}. Named outputs can be selected using the - * {@link #select} function. - * - * @param - * The type of the output. - */ -public class SplitDataStream { - - DataStream dataStream; - - protected SplitDataStream(DataStream dataStream) { - this.dataStream = dataStream.copy(); - } +package org.apache.flink.streaming.api.datastream; + +import java.util.Arrays; + +import org.apache.flink.streaming.api.collector.OutputSelector; + +/** + * The SplitDataStream represents an operator that has been split using an + * {@link OutputSelector}. Named outputs can be selected using the + * {@link #select} function. To apply transformation on the whole output simply + * call the transformation on the SplitDataStream + * + * @param + * The type of the output. + */ +public class SplitDataStream extends DataStream { + + protected SplitDataStream(DataStream 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 getOutputType() { - return dataStream.getType(); + public DataStream select(String... outputNames) { + return selectOutput(outputNames); + } + + private DataStream selectOutput(String[] outputNames) { + DataStream 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 select(String... outputNames) { - return selectOutput(outputNames); - } - - /** - * Selects all output names from a split data stream. - * - * @return Returns the selected DataStream - */ - public DataStream selectAll() { - DataStream returnStream = dataStream.copy(); - returnStream.selectAll = true; - return returnStream; - } - - private DataStream selectOutput(String[] outputNames) { - DataStream returnStream = dataStream.copy(); - returnStream.userDefinedNames = Arrays.asList(outputNames); - return returnStream; - } - -} + +} diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java index b7a3d4457d90b322e22676ebf1b2614740b56a13..78cbbe5241270bf9f2b6270dc448c78adfead1ca 100644 --- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java +++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java @@ -100,7 +100,7 @@ public class DirectedOutputTest { 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.selectAll().addSink(new ListSink(ALL)); + source.addSink(new ListSink(ALL)); env.execute(); assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), outputs.get(EVEN)); diff --git a/flink-addons/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala b/flink-addons/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala index a4156a15464d458fc0690f78111721bc59f1af5d..9e33f8072458a2d075ef8c6cd1a50824f5b34d55 100644 --- a/flink-addons/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala +++ b/flink-addons/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala @@ -23,26 +23,17 @@ import org.apache.flink.streaming.api.datastream.{ SplitDataStream => SplitJavaS /** * The SplitDataStream represents an operator that has been split using an * {@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 * The type of the output. */ -class SplitDataStream[T](javaStream: SplitJavaStream[T]) { - - /** - * Gets the underlying java DataStream object. - */ - private[flink] def getJavaStream: SplitJavaStream[T] = javaStream +class SplitDataStream[T](javaStream: SplitJavaStream[T]) extends DataStream[T](javaStream){ /** * Sets the output names for which the next operator will receive values. */ def select(outputNames: String*): DataStream[T] = javaStream.select(outputNames: _*) - - /** - * Selects all output names from a split data stream. - */ - def selectAll(): DataStream[T] = javaStream.selectAll() - + }