Skip to content

Commit

Permalink
[FLINK-34543][docs] Add document of full partition processing on non-…
Browse files Browse the repository at this point in the history
…keyed datastream

Signed-off-by: Xu Huang <zuosi.hx@alibaba-inc.com>
  • Loading branch information
Xu Huang authored and reswqa committed Jul 18, 2024
1 parent acc3486 commit 255abc7
Show file tree
Hide file tree
Showing 4 changed files with 262 additions and 0 deletions.
104 changes: 104 additions & 0 deletions docs/content.zh/docs/dev/datastream/operators/full_window_partition.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
---
title: "Full Window Partition"
weight: 5
type: docs
aliases:
- /dev/stream/operators/full_window_partition.html
---
<!--
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.
-->

# Full Window Partition Processing on DataStream

This page explains the use of full window partition processing API on DataStream.
Flink enables both keyed and non-keyed DataStream to directly transform into
`PartitionWindowedStream` now.
The `PartitionWindowedStream` represents collecting all records of each subtask separately
into a full window.
The `PartitionWindowedStream` support four APIs: `mapPartition`, `sortPartition`, `aggregate`
and `reduce`.

Note: Details about the design and implementation of the full window partition processing can be
found in the proposal and design document
[FLIP-380: Support Full Partition Processing On Non-keyed DataStream](https://cwiki.apache.org/confluence/display/FLINK/FLIP-380%3A+Support+Full+Partition+Processing+On+Non-keyed+DataStream).

## MapPartition

`MapPartition` represents collecting all records of each subtask separately into a full window
and process them using the given `MapPartitionFunction` within each subtask. The
`MapPartitionFunction` is called at the end of inputs.

An example of calculating the sum of the elements in each subtask is as follows:

```java
DataStream<Integer> dataStream = //...
PartitionWindowedStream<Integer> partitionWindowedDataStream = dataStream.fullWindowPartition();

DataStream<Integer> resultStream = partitionWindowedDataStream.mapPartition(
new MapPartitionFunction<Integer, Integer>() {
@Override
public void mapPartition(
Iterable<Integer> values, Collector<Integer> out) {
int result = 0;
for (Integer value : values) {
result += value;
}
out.collect(result);
}
}
);
```

## SortPartition
`SortPartition` represents collecting all records of each subtask separately into a full window
and sorts them by the given record comparator in each subtask at the end of inputs.

An example of sorting the records by the first element of tuple in each subtask is as follows:

```java
DataStream<Tuple2<Integer, Integer>> dataStream = //...
PartitionWindowedStream<Tuple2<Integer, Integer>> partitionWindowedDataStream = dataStream.fullWindowPartition();
DataStream<Integer> resultStream = partitionWindowedDataStream.sortPartition(0, Order.ASCENDING);
```

## Aggregate
`Aggregate` represents collecting all records of each subtask separately into a full window and
applies the given `AggregateFunction` to the records of the window. The `AggregateFunction`
is called for each element, aggregating values incrementally within the window.

An example of aggregate the records in each subtask is as follows:

```java
DataStream<Tuple2<Integer, Integer>> dataStream = //...
PartitionWindowedStream<Tuple2<Integer, Integer>> partitionWindowedDataStream = dataStream.fullWindowPartition();
DataStream<Integer> resultStream = partitionWindowedDataStream.aggregate(new AggregateFunction<>{...});
```

## Reduce
`Reduce` represents applies a reduce transformation on all the records in the partition.
The `ReduceFunction` will be called for every record in the window.
An example is as follows:

```java
DataStream<Tuple2<Integer, Integer>> dataStream = //...
PartitionWindowedStream<Tuple2<Integer, Integer>> partitionWindowedDataStream = dataStream.fullWindowPartition();
DataStream<Integer> resultStream = partitionWindowedDataStream.aggregate(new ReduceFunction<>{...});
```

{{< top >}}
26 changes: 26 additions & 0 deletions docs/content.zh/docs/dev/datastream/operators/overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -612,6 +612,32 @@ env.execute()
{{< /tab >}}
{{< /tabs>}}

### Full Window Partition
#### DataStream &rarr; PartitionWindowedStream

将所有的数据记录收集到一个Window中,然后在输入数据流结束时按 partition 进行处理。本方法特别适用于批处理场景。
对于非 Keyed DataStream,一个 partition 包含一个并行子任务的所有数据记录。
对于 Keyed DataStream,一个 partition 包含所有具有相同 key 的数据记录。

```java
DataStream<Integer> dataStream = //...
PartitionWindowedStream<Integer> partitionWindowedDataStream = dataStream.fullWindowPartition();
// do full window partition processing with PartitionWindowedStream
DataStream<Integer> resultStream = partitionWindowedDataStream.mapPartition(
new MapPartitionFunction<Integer, Integer>() {
@Override
public void mapPartition(
Iterable<Integer> values, Collector<Integer> out) {
int result = 0;
for (Integer value : values) {
result += value;
}
out.collect(result);
}
}
);
```

## 物理分区

Flink 也提供以下方法让用户根据需要在数据转换完成后对数据分区进行更细粒度的配置。
Expand Down
104 changes: 104 additions & 0 deletions docs/content/docs/dev/datastream/operators/full_window_partition.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
---
title: "Full Window Partition"
weight: 5
type: docs
aliases:
- /dev/stream/operators/full_window_partition.html
---
<!--
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.
-->

# Full Window Partition Processing on DataStream

This page explains the use of full window partition processing API on DataStream.
Flink enables both keyed and non-keyed DataStream to directly transform into
`PartitionWindowedStream` now.
The `PartitionWindowedStream` represents collecting all records of each subtask separately
into a full window.
The `PartitionWindowedStream` support four APIs: `mapPartition`, `sortPartition`, `aggregate`
and `reduce`.

Note: Details about the design and implementation of the full window partition processing can be
found in the proposal and design document
[FLIP-380: Support Full Partition Processing On Non-keyed DataStream](https://cwiki.apache.org/confluence/display/FLINK/FLIP-380%3A+Support+Full+Partition+Processing+On+Non-keyed+DataStream).

## MapPartition

`MapPartition` represents collecting all records of each subtask separately into a full window
and process them using the given `MapPartitionFunction` within each subtask. The
`MapPartitionFunction` is called at the end of inputs.

An example of calculating the sum of the elements in each subtask is as follows:

```java
DataStream<Integer> dataStream = //...
PartitionWindowedStream<Integer> partitionWindowedDataStream = dataStream.fullWindowPartition();

DataStream<Integer> resultStream = partitionWindowedDataStream.mapPartition(
new MapPartitionFunction<Integer, Integer>() {
@Override
public void mapPartition(
Iterable<Integer> values, Collector<Integer> out) {
int result = 0;
for (Integer value : values) {
result += value;
}
out.collect(result);
}
}
);
```

## SortPartition
`SortPartition` represents collecting all records of each subtask separately into a full window
and sorts them by the given record comparator in each subtask at the end of inputs.

An example of sorting the records by the first element of tuple in each subtask is as follows:

```java
DataStream<Tuple2<Integer, Integer>> dataStream = //...
PartitionWindowedStream<Tuple2<Integer, Integer>> partitionWindowedDataStream = dataStream.fullWindowPartition();
DataStream<Integer> resultStream = partitionWindowedDataStream.sortPartition(0, Order.ASCENDING);
```

## Aggregate
`Aggregate` represents collecting all records of each subtask separately into a full window and
applies the given `AggregateFunction` to the records of the window. The `AggregateFunction`
is called for each element, aggregating values incrementally within the window.

An example of aggregate the records in each subtask is as follows:

```java
DataStream<Tuple2<Integer, Integer>> dataStream = //...
PartitionWindowedStream<Tuple2<Integer, Integer>> partitionWindowedDataStream = dataStream.fullWindowPartition();
DataStream<Integer> resultStream = partitionWindowedDataStream.aggregate(new AggregateFunction<>{...});
```

## Reduce
`Reduce` represents applies a reduce transformation on all the records in the partition.
The `ReduceFunction` will be called for every record in the window.
An example is as follows:

```java
DataStream<Tuple2<Integer, Integer>> dataStream = //...
PartitionWindowedStream<Tuple2<Integer, Integer>> partitionWindowedDataStream = dataStream.fullWindowPartition();
DataStream<Integer> resultStream = partitionWindowedDataStream.aggregate(new ReduceFunction<>{...});
```

{{< top >}}
28 changes: 28 additions & 0 deletions docs/content/docs/dev/datastream/operators/overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -617,6 +617,34 @@ env.execute()
{{< /tab >}}
{{< /tabs>}}

### Full Window Partition
#### DataStream &rarr; PartitionWindowedStream

Collects all records of each partition separately into a full window and processes them. The window
emission will be triggered at the end of inputs.
This approach is primarily applicable to batch processing scenarios.
For non-keyed DataStream, a partition contains all records of a subtask.
For KeyedStream, a partition contains all records of a key.

```java
DataStream<Integer> dataStream = //...
PartitionWindowedStream<Integer> partitionWindowedDataStream = dataStream.fullWindowPartition();
// do full window partition processing with PartitionWindowedStream
DataStream<Integer> resultStream = partitionWindowedDataStream.mapPartition(
new MapPartitionFunction<Integer, Integer>() {
@Override
public void mapPartition(
Iterable<Integer> values, Collector<Integer> out) {
int result = 0;
for (Integer value : values) {
result += value;
}
out.collect(result);
}
}
);
```

## Physical Partitioning

Flink also gives low-level control (if desired) on the exact stream partitioning after a transformation, via the following functions.
Expand Down

0 comments on commit 255abc7

Please sign in to comment.