Skip to content

Commit

Permalink
[FLINK-8571] [DataStream] Introduce utility function that reinterpret…
Browse files Browse the repository at this point in the history
…s a data stream as keyed stream

This closes apache#5424.
  • Loading branch information
StefanRRichter committed Feb 9, 2018
1 parent bfe6f84 commit 91eea37
Show file tree
Hide file tree
Showing 8 changed files with 468 additions and 7 deletions.
1 change: 0 additions & 1 deletion docs/dev/api_concepts.md
Original file line number Diff line number Diff line change
Expand Up @@ -895,4 +895,3 @@ result type ```R``` for the final result. E.g. for a histogram, ```V``` is a num
a histogram. ```SimpleAccumulator``` is for the cases where both types are the same, e.g. for counters.

{% top %}

79 changes: 79 additions & 0 deletions docs/dev/stream/experimental.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
---
title: "Experimental Features"
nav-id: experimental_features
nav-show_overview: true
nav-parent_id: streaming
nav-pos: 100
---
<!--
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
https://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.
-->

This section describes experimental features in the DataStream API. Experimental features are still evolving and can be either unstable,
incomplete, or subject to heavy change in future versions.

Reinterpreting a pre-partitioned data stream as keyed stream
------------------------------------------------------------

We can re-interpret a pre-partitioned data stream as a keyed stream to avoid shuffling.

**WARNING**: The re-interpreted data stream **MUST** already be pre-partitioned in **EXACTLY** the same way Flink's keyBy would partition
the data in a shuffle w.r.t. key-group assignment.

One use-case for this could be a materialized shuffle between two jobs: the first job performs a keyBy shuffle and materializes
each output into a partition. A second job has sources that, for each parallel instance, reads from the corresponding partitions
created by the first job. Those sources can now be re-interpreted as keyed streams, e.g. to apply windowing. Notice that this trick
makes the second job embarrassingly parallel, which can be helpful for a fine-grained recovery scheme.

This re-interpretation functionality is exposed through `DataStreamUtils`:

{% highlight java %}
static <T, K> KeyedStream<T, K> reinterpretAsKeyedStream(
DataStream<T> stream,
KeySelector<T, K> keySelector,
TypeInformation<K> typeInfo)
{% endhighlight %}

Given a base stream, a key selector, and type information,
the method creates a keyed stream from the base stream.

Code example:

<div class="codetabs" markdown="1">
<div data-lang="java" markdown="1">
{% highlight java %}
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStreamSource<Integer> source = ...
DataStreamUtils.reinterpretAsKeyedStream(source, (in) -> in, TypeInformation.of(Integer.class))
.timeWindow(Time.seconds(1))
.reduce((a, b) -> a + b)
.addSink(new DiscardingSink<>());
env.execute();
{% endhighlight %}
</div>
<div data-lang="scala" markdown="1">
{% highlight scala %}
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setParallelism(1)
val source = ...
new DataStreamUtils(source).reinterpretAsKeyedStream((in) => in)
.timeWindow(Time.seconds(1))
.reduce((a, b) => a + b)
.addSink(new DiscardingSink[Int])
env.execute()
{% endhighlight %}
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,18 @@
package org.apache.flink.streaming.api.datastream;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.runtime.net.ConnectionUtils;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.experimental.CollectSink;
import org.apache.flink.streaming.experimental.SocketStreamIterator;
import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;

import java.io.IOException;
import java.net.InetAddress;
Expand All @@ -33,7 +38,7 @@
import java.util.Iterator;

/**
* A collection of experimental utilities for {@link DataStream DataStreams}.
* A collection of utilities for {@link DataStream DataStreams}.
*
* <p>This experimental class is relocated from flink-streaming-contrib. Please see package-info.java
* for more information.
Expand Down Expand Up @@ -85,6 +90,61 @@ public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream) throws IOExcep
return iter;
}

/**
* Reinterprets the given {@link DataStream} as a {@link KeyedStream}, which extracts keys with the given
* {@link KeySelector}.
*
* <p>IMPORTANT: For every partition of the base stream, the keys of events in the base stream must be
* partitioned exactly in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}.
*
* @param stream The data stream to reinterpret. For every partition, this stream must be partitioned exactly
* in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}.
* @param keySelector Function that defines how keys are extracted from the data stream.
* @param <T> Type of events in the data stream.
* @param <K> Type of the extracted keys.
* @return The reinterpretation of the {@link DataStream} as a {@link KeyedStream}.
*/
public static <T, K> KeyedStream<T, K> reinterpretAsKeyedStream(
DataStream<T> stream,
KeySelector<T, K> keySelector) {

return reinterpretAsKeyedStream(
stream,
keySelector,
TypeExtractor.getKeySelectorTypes(keySelector, stream.getType()));
}

/**
* Reinterprets the given {@link DataStream} as a {@link KeyedStream}, which extracts keys with the given
* {@link KeySelector}.
*
* <p>IMPORTANT: For every partition of the base stream, the keys of events in the base stream must be
* partitioned exactly in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}.
*
* @param stream The data stream to reinterpret. For every partition, this stream must be partitioned exactly
* in the same way as if it was created through a {@link DataStream#keyBy(KeySelector)}.
* @param keySelector Function that defines how keys are extracted from the data stream.
* @param typeInfo Explicit type information about the key type.
* @param <T> Type of events in the data stream.
* @param <K> Type of the extracted keys.
* @return The reinterpretation of the {@link DataStream} as a {@link KeyedStream}.
*/
public static <T, K> KeyedStream<T, K> reinterpretAsKeyedStream(
DataStream<T> stream,
KeySelector<T, K> keySelector,
TypeInformation<K> typeInfo) {

PartitionTransformation<T> partitionTransformation = new PartitionTransformation<>(
stream.getTransformation(),
new ForwardPartitioner<>());

return new KeyedStream<>(
stream,
partitionTransformation,
keySelector,
typeInfo);
}

private static class CallExecute extends Thread {

private final StreamExecutionEnvironment toTrigger;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,12 +120,37 @@ public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
* Function for determining state partitions
*/
public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector, TypeInformation<KEY> keyType) {
super(
dataStream.getExecutionEnvironment(),
this(
dataStream,
new PartitionTransformation<>(
dataStream.getTransformation(),
new KeyGroupStreamPartitioner<>(keySelector, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM)));
this.keySelector = keySelector;
new KeyGroupStreamPartitioner<>(keySelector, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM)),
keySelector,
keyType);
}

/**
* Creates a new {@link KeyedStream} using the given {@link KeySelector} and {@link TypeInformation}
* to partition operator state by key, where the partitioning is defined by a {@link PartitionTransformation}.
*
* @param stream
* Base stream of data
* @param partitionTransformation
* Function that determines how the keys are distributed to downstream operator(s)
* @param keySelector
* Function to extract keys from the base stream
* @param keyType
* Defines the type of the extracted keys
*/
@Internal
KeyedStream(
DataStream<T> stream,
PartitionTransformation<T> partitionTransformation,
KeySelector<T, KEY> keySelector,
TypeInformation<KEY> keyType) {

super(stream.getExecutionEnvironment(), partitionTransformation);
this.keySelector = clean(keySelector);
this.keyType = validateKeyType(keyType);
}

Expand Down
Loading

0 comments on commit 91eea37

Please sign in to comment.