Return-Path: X-Original-To: apmail-flink-commits-archive@minotaur.apache.org Delivered-To: apmail-flink-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 815E910151 for ; Mon, 12 Jan 2015 10:20:10 +0000 (UTC) Received: (qmail 11109 invoked by uid 500); 12 Jan 2015 10:20:11 -0000 Delivered-To: apmail-flink-commits-archive@flink.apache.org Received: (qmail 11077 invoked by uid 500); 12 Jan 2015 10:20:11 -0000 Mailing-List: contact commits-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list commits@flink.apache.org Received: (qmail 11068 invoked by uid 99); 12 Jan 2015 10:20:11 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 12 Jan 2015 10:20:11 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 357999AB137; Mon, 12 Jan 2015 10:20:11 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: mbalassi@apache.org To: commits@flink.apache.org Message-Id: <0481b4c56de142599a9f5fda216c677e@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: flink git commit: [FLINK-1380] [streaming] Updated SplitDataStream to extend DataStream to get rid of selectAll method for splits Date: Mon, 12 Jan 2015 10:20:11 +0000 (UTC) Repository: flink Updated Branches: refs/heads/master d908ca197 -> 9f07373f3 [FLINK-1380] [streaming] Updated SplitDataStream to extend DataStream to get rid of selectAll method for splits Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9f07373f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9f07373f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9f07373f Branch: refs/heads/master Commit: 9f07373f358f4a723a170a4f91abaa0b8bf8d62a Parents: d908ca1 Author: Gyula Fora Authored: Fri Jan 9 16:52:46 2015 +0100 Committer: mbalassi Committed: Mon Jan 12 09:59:14 2015 +0100 ---------------------------------------------------------------------- docs/streaming_guide.md | 2 +- .../streaming/api/datastream/DataStream.java | 2 +- .../datastream/SingleOutputStreamOperator.java | 27 +++-- .../api/datastream/SplitDataStream.java | 121 ++++++++----------- .../api/collector/DirectedOutputTest.java | 2 +- .../streaming/api/scala/SplitDataStream.scala | 17 +-- 6 files changed, 77 insertions(+), 94 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/9f07373f/docs/streaming_guide.md ---------------------------------------------------------------------- diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md index 7808b9e..1b0a1e9 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): http://git-wip-us.apache.org/repos/asf/flink/blob/9f07373f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java ---------------------------------------------------------------------- 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 8e21218..7d1659f 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>(); http://git-wip-us.apache.org/repos/asf/flink/blob/9f07373f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java ---------------------------------------------------------------------- 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 aa85579..5a8261e 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); } /** http://git-wip-us.apache.org/repos/asf/flink/blob/9f07373f/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java ---------------------------------------------------------------------- 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 4fac04c..1bf4f9c 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; - } - -} + +} http://git-wip-us.apache.org/repos/asf/flink/blob/9f07373f/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java ---------------------------------------------------------------------- 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 b7a3d44..78cbbe5 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)); http://git-wip-us.apache.org/repos/asf/flink/blob/9f07373f/flink-addons/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala ---------------------------------------------------------------------- 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 a4156a1..9e33f80 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() - + }