Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 4151dde

Browse files
authored
remove processContext usage across examples (java and kotlin) (#37937)
* add license
1 parent a615657 commit 4151dde

79 files changed

Lines changed: 1062 additions & 547 deletions

File tree

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.
Lines changed: 207 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,207 @@
1+
---
2+
# Licensed to the Apache Software Foundation (ASF) under one
3+
# or more contributor license agreements. See the NOTICE file
4+
# distributed with this work for additional information
5+
# regarding copyright ownership. The ASF licenses this file
6+
# to you under the Apache License, Version 2.0 (the
7+
# "License"); you may not use this file except in compliance
8+
# with the License. You may obtain a copy of the License at
9+
#
10+
# http://www.apache.org/licenses/LICENSE-2.0
11+
#
12+
# Unless required by applicable law or agreed to in writing,
13+
# software distributed under the License is distributed on an
14+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
# KIND, either express or implied. See the License for the
16+
# specific language governing permissions and limitations
17+
# under the License.
18+
---
19+
name: beam-dofn-modernizer
20+
description: Rewrite Apache Beam DoFn methods (@ProcessElement, @OnTimer, @OnWindowExpiration) to remove legacy ProcessContext or OnTimerContext usage. Use this skill when you encounter DoFn methods that use context.element(), context.output(), etc., and need to modernize them using parameter injection (@Element, @Timestamp, @Pane, OutputReceiver, MultiOutputReceiver).
21+
---
22+
23+
# Modernizing Apache Beam DoFns
24+
25+
Apache Beam has moved towards parameter injection in `DoFn` methods to improve readability and allow for more efficient execution. This skill helps you migrate legacy `ProcessContext` and `OnTimerContext` usage to modern annotated parameters.
26+
27+
## Core Mappings
28+
29+
When rewriting a `@ProcessElement` or `@OnTimer` method, replace the context argument with the corresponding parameters based on the usage:
30+
31+
| Legacy Context Usage (e.g. `ProcessContext c`) | Modern Parameter Replacement |
32+
| :--- | :--- |
33+
| `c.element()` | `@Element T element` |
34+
| `c.timestamp()` | `@Timestamp Instant timestamp` |
35+
| `c.pane()` | `PaneInfo pane` |
36+
| `c.window()` | `BoundedWindow window` |
37+
| `c.sideInput(PCollectionView<T> view)` | `@SideInput("viewName") T value` |
38+
| `c.getPipelineOptions()` | `PipelineOptions options` |
39+
| `c.output(value)` | `OutputReceiver<T> receiver` then `receiver.output(value)` |
40+
| `c.output(tag, value)` | `MultiOutputReceiver receiver` then `receiver.get(tag).output(value)` |
41+
| `c.outputWithTimestamp(value, ts)` | `OutputReceiver<T> receiver` then `receiver.outputWithTimestamp(value, ts)` |
42+
43+
## Method Signature Changes
44+
45+
### @ProcessElement
46+
47+
**Legacy:**
48+
```java
49+
@ProcessElement
50+
public void processElement(ProcessContext c) {
51+
T element = c.element();
52+
c.output(transform(element));
53+
}
54+
```
55+
56+
**Modern:**
57+
```java
58+
@ProcessElement
59+
public void processElement(
60+
@Element T element,
61+
@Timestamp Instant timestamp,
62+
OutputReceiver<V> receiver) {
63+
receiver.output(transform(element));
64+
}
65+
```
66+
67+
### @OnTimer
68+
69+
**Legacy:**
70+
```java
71+
@OnTimer("timerId")
72+
public void onTimer(OnTimerContext c) {
73+
c.output(someValue);
74+
}
75+
```
76+
77+
**Modern:**
78+
```java
79+
@OnTimer("timerId")
80+
public void onTimer(
81+
@Timestamp Instant timestamp,
82+
BoundedWindow window,
83+
OutputReceiver<V> receiver) {
84+
receiver.output(someValue);
85+
}
86+
```
87+
88+
## Best Practices
89+
90+
1. **Specific OutputReceiver**: If the method only outputs to the main output, use `OutputReceiver<T>`. If it outputs to multiple tags, use `MultiOutputReceiver`.
91+
2. **Element Type**: Ensure the `@Element` parameter type matches the input type of the `DoFn`.
92+
3. **Imports**: Don't forget to add imports for:
93+
* `org.apache.beam.sdk.transforms.DoFn.Element`
94+
* `org.apache.beam.sdk.transforms.DoFn.Timestamp`
95+
* `org.apache.beam.sdk.transforms.DoFn.OutputReceiver`
96+
* `org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver` (if needed)
97+
* `org.apache.beam.sdk.values.PCollectionView` (if using `@SideInput`)
98+
* `org.apache.beam.sdk.transforms.DoFn.SideInput`
99+
* `org.apache.beam.sdk.transforms.windowing.PaneInfo`
100+
4. **Side Inputs**: When using `@SideInput`, make sure to use the correct name that matches the one passed to `ParDo.withSideInput("name", view)`.
101+
5. **Parameter Naming and Redundant Variables**: Use descriptive names for the `@Element` parameter (e.g., `record`, `line`, `row`) instead of a generic `element` if it improves readability. Do not create a redundant local variable to copy the element (e.g., `MyType elm = element;`), use the parameter directly.
102+
103+
## Example Conversion
104+
105+
### Before:
106+
```java
107+
@ProcessElement
108+
public void processElement(ProcessContext c) {
109+
KV<String, Integer> element = c.element();
110+
Instant ts = c.timestamp();
111+
if (element.getValue() > threshold) {
112+
c.output(element.getKey());
113+
c.output(specialTag, element.getValue());
114+
}
115+
}
116+
```
117+
118+
### After:
119+
```java
120+
@ProcessElement
121+
public void processElement(
122+
@Element KV<String, Integer> element,
123+
@Timestamp Instant timestamp,
124+
MultiOutputReceiver receiver) {
125+
if (element.getValue() > threshold) {
126+
receiver.get(mainTag).output(element.getKey());
127+
receiver.get(specialTag).output(element.getValue());
128+
}
129+
}
130+
```
131+
> [!NOTE]
132+
> If you only have one output, use `OutputReceiver<String> receiver` and `receiver.output(element.getKey())`.
133+
134+
## Side Input Conversion
135+
136+
Modernizing side inputs involves removing the `PCollectionView` from the `DoFn` constructor and using `@SideInput` parameter injection instead.
137+
138+
### Before (Legacy):
139+
140+
**PTransform/Pipeline side:**
141+
```java
142+
PCollectionView<String> myView = ...;
143+
input.apply(ParDo.of(new MyFn(myView)).withSideInputs(myView));
144+
```
145+
146+
**DoFn side:**
147+
```java
148+
class MyFn extends DoFn<T, V> {
149+
private final PCollectionView<String> view;
150+
MyFn(PCollectionView<String> view) { this.view = view; }
151+
152+
@ProcessElement
153+
public void processElement(ProcessContext c) {
154+
String value = c.sideInput(view);
155+
// ...
156+
}
157+
}
158+
```
159+
160+
### Nullable Side Inputs
161+
162+
If a side input is optional and a `DoFn` has conditional logic based on whether the side input is present, it is best to split the `DoFn` into two separate classes: one that requires the side input and one that does not. This avoids creating complex, conditional `DoFn`s and ensures type safety.
163+
164+
**PTransform/Pipeline side:**
165+
```java
166+
PCollectionView<String> myView = ...;
167+
input.apply(ParDo.of(new MyFn(myView)).withSideInputs(myView));
168+
//or
169+
input.apply(ParDo.of(new MyFn(null))); // to introduce null
170+
```
171+
**DoFn side:**
172+
```java
173+
class MyFn extends DoFn<T, V> {
174+
private final PCollectionView<String> view;
175+
MyFn(PCollectionView<String> view) { this.view = view; }
176+
177+
@ProcessElement
178+
public void processElement(ProcessContext c) {
179+
String value = null;
180+
if (this.view != null) { // can do conditional side input
181+
value = c.sideInput(this.view);
182+
}
183+
184+
// ...
185+
}
186+
}
187+
```
188+
189+
### After (Modern):
190+
191+
**PTransform/Pipeline side:**
192+
```java
193+
PCollectionView<String> myView = ...;
194+
input.apply(ParDo.of(new MyFn()).withSideInput("sideInputName", myView));
195+
```
196+
197+
**DoFn side:**
198+
```java
199+
class MyFn extends DoFn<T, V> {
200+
@ProcessElement
201+
public void processElement(
202+
@Element T element,
203+
@SideInput("sideInputName") String value) {
204+
// value is injected directly
205+
}
206+
}
207+
```

examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergBatchWriteExample.java

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,8 @@
2828
import org.apache.beam.sdk.options.Validation;
2929
import org.apache.beam.sdk.schemas.Schema;
3030
import org.apache.beam.sdk.transforms.DoFn;
31+
import org.apache.beam.sdk.transforms.DoFn.Element;
32+
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
3133
import org.apache.beam.sdk.transforms.MapElements;
3234
import org.apache.beam.sdk.transforms.PTransform;
3335
import org.apache.beam.sdk.transforms.ParDo;
@@ -74,9 +76,8 @@ private static Row flattenAnalyticsRow(Row row) {
7476

7577
static class ExtractBrowserTransactionsFn extends DoFn<Row, KV<String, Long>> {
7678
@ProcessElement
77-
public void processElement(ProcessContext c) {
78-
Row row = c.element();
79-
c.output(
79+
public void processElement(@Element Row row, OutputReceiver<KV<String, Long>> receiver) {
80+
receiver.output(
8081
KV.of(
8182
Preconditions.checkStateNotNull(row.getString("browser")),
8283
Preconditions.checkStateNotNull(row.getInt64("transactions"))));
@@ -85,13 +86,13 @@ public void processElement(ProcessContext c) {
8586

8687
static class FormatCountsFn extends DoFn<KV<String, Long>, Row> {
8788
@ProcessElement
88-
public void processElement(ProcessContext c) {
89+
public void processElement(@Element KV<String, Long> element, OutputReceiver<Row> receiver) {
8990
Row row =
9091
Row.withSchema(AGGREGATED_SCHEMA)
91-
.withFieldValue("browser", c.element().getKey())
92-
.withFieldValue("transaction_count", c.element().getValue())
92+
.withFieldValue("browser", element.getKey())
93+
.withFieldValue("transaction_count", element.getValue())
9394
.build();
94-
c.output(row);
95+
receiver.output(row);
9596
}
9697
}
9798

examples/java/sql/src/main/java/org/apache/beam/examples/SchemaTransformExample.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,8 @@
4242
import org.apache.beam.sdk.schemas.transforms.Select;
4343
import org.apache.beam.sdk.transforms.Create;
4444
import org.apache.beam.sdk.transforms.DoFn;
45+
import org.apache.beam.sdk.transforms.DoFn.Element;
46+
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
4547
import org.apache.beam.sdk.transforms.Max;
4648
import org.apache.beam.sdk.transforms.Min;
4749
import org.apache.beam.sdk.transforms.ParDo;
@@ -101,9 +103,9 @@ public LogOutput(String prefix) {
101103
}
102104

103105
@ProcessElement
104-
public void processElement(ProcessContext c) throws Exception {
105-
LOG.info("{}{}", prefix, c.element());
106-
c.output(c.element());
106+
public void processElement(@Element T element, OutputReceiver<T> receiver) throws Exception {
107+
LOG.info("{}{}", prefix, element);
108+
receiver.output(element);
107109
}
108110
}
109111
}

examples/java/sql/src/main/java/org/apache/beam/examples/SqlTransformExample.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,8 @@
4141
import org.apache.beam.sdk.schemas.Schema;
4242
import org.apache.beam.sdk.transforms.Create;
4343
import org.apache.beam.sdk.transforms.DoFn;
44+
import org.apache.beam.sdk.transforms.DoFn.Element;
45+
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
4446
import org.apache.beam.sdk.transforms.ParDo;
4547
import org.apache.beam.sdk.values.PCollection;
4648
import org.apache.beam.sdk.values.Row;
@@ -95,9 +97,9 @@ public LogOutput(String prefix) {
9597
}
9698

9799
@ProcessElement
98-
public void processElement(ProcessContext c) throws Exception {
99-
LOG.info("{}{}", prefix, c.element());
100-
c.output(c.element());
100+
public void processElement(@Element T element, OutputReceiver<T> receiver) throws Exception {
101+
LOG.info("{}{}", prefix, element);
102+
receiver.output(element);
101103
}
102104
}
103105
}

examples/java/src/main/java/org/apache/beam/examples/ApproximateQuantilesExample.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,8 @@
2424
import org.apache.beam.sdk.transforms.ApproximateQuantiles;
2525
import org.apache.beam.sdk.transforms.Create;
2626
import org.apache.beam.sdk.transforms.DoFn;
27+
import org.apache.beam.sdk.transforms.DoFn.Element;
28+
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
2729
import org.apache.beam.sdk.transforms.ParDo;
2830
import org.apache.beam.sdk.values.PCollection;
2931
import org.slf4j.Logger;
@@ -70,9 +72,9 @@ public LogOutput(String prefix) {
7072
}
7173

7274
@ProcessElement
73-
public void processElement(ProcessContext c) throws Exception {
74-
LOG.info("{}{}", prefix, c.element());
75-
c.output(c.element());
75+
public void processElement(@Element T element, OutputReceiver<T> receiver) throws Exception {
76+
LOG.info("{}{}", prefix, element);
77+
receiver.output(element);
7678
}
7779
}
7880
}

examples/java/src/main/java/org/apache/beam/examples/CoCombineTransformExample.java

Lines changed: 11 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,8 @@
4646
import org.apache.beam.sdk.transforms.CombineFns;
4747
import org.apache.beam.sdk.transforms.Create;
4848
import org.apache.beam.sdk.transforms.DoFn;
49+
import org.apache.beam.sdk.transforms.DoFn.Element;
50+
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
4951
import org.apache.beam.sdk.transforms.Max;
5052
import org.apache.beam.sdk.transforms.Min;
5153
import org.apache.beam.sdk.transforms.ParDo;
@@ -185,13 +187,16 @@ public Long apply(Long input) {
185187
new DoFn<
186188
KV<Long, CombineFns.CoCombineResult>, KV<Long, Iterable<KV<String, Long>>>>() {
187189
@ProcessElement
188-
public void processElement(ProcessContext c) throws Exception {
189-
CombineFns.CoCombineResult e = c.element().getValue();
190+
public void processElement(
191+
@Element KV<Long, CombineFns.CoCombineResult> element,
192+
OutputReceiver<KV<Long, Iterable<KV<String, Long>>>> receiver)
193+
throws Exception {
194+
CombineFns.CoCombineResult e = element.getValue();
190195
ArrayList<KV<String, Long>> o = new ArrayList<KV<String, Long>>();
191196
o.add(KV.of(minTag.getId(), e.get(minTag)));
192197
o.add(KV.of(maxTag.getId(), e.get(maxTag)));
193198
o.add(KV.of(sumTag.getId(), e.get(sumTag)));
194-
c.output(KV.of(c.element().getKey(), o));
199+
receiver.output(KV.of(element.getKey(), o));
195200
}
196201
}));
197202

@@ -210,9 +215,9 @@ public LogOutput(String prefix) {
210215
}
211216

212217
@ProcessElement
213-
public void processElement(ProcessContext c) throws Exception {
214-
LOG.info("{}{}", prefix, c.element());
215-
c.output(c.element());
218+
public void processElement(@Element T element, OutputReceiver<T> receiver) throws Exception {
219+
LOG.info("{}{}", prefix, element);
220+
receiver.output(element);
216221
}
217222
}
218223
}

examples/java/src/main/java/org/apache/beam/examples/CoGroupByKeyExample.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,8 @@
2222
import org.apache.beam.sdk.options.PipelineOptionsFactory;
2323
import org.apache.beam.sdk.transforms.Create;
2424
import org.apache.beam.sdk.transforms.DoFn;
25+
import org.apache.beam.sdk.transforms.DoFn.Element;
26+
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
2527
import org.apache.beam.sdk.transforms.ParDo;
2628
import org.apache.beam.sdk.transforms.join.CoGbkResult;
2729
import org.apache.beam.sdk.transforms.join.CoGroupByKey;
@@ -84,9 +86,9 @@ public LogOutput(String prefix) {
8486
}
8587

8688
@ProcessElement
87-
public void processElement(ProcessContext c) throws Exception {
88-
LOG.info("{}{}", prefix, c.element());
89-
c.output(c.element());
89+
public void processElement(@Element T element, OutputReceiver<T> receiver) throws Exception {
90+
LOG.info("{}{}", prefix, element);
91+
receiver.output(element);
9092
}
9193
}
9294
}

examples/java/src/main/java/org/apache/beam/examples/CombineExample.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,8 @@
2323
import org.apache.beam.sdk.transforms.Combine;
2424
import org.apache.beam.sdk.transforms.Create;
2525
import org.apache.beam.sdk.transforms.DoFn;
26+
import org.apache.beam.sdk.transforms.DoFn.Element;
27+
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
2628
import org.apache.beam.sdk.transforms.ParDo;
2729
import org.apache.beam.sdk.transforms.Sum;
2830
import org.apache.beam.sdk.values.PCollection;
@@ -68,9 +70,9 @@ public LogOutput(String prefix) {
6870
}
6971

7072
@ProcessElement
71-
public void processElement(ProcessContext c) throws Exception {
72-
LOG.info("{}{}", prefix, c.element());
73-
c.output(c.element());
73+
public void processElement(@Element T element, OutputReceiver<T> receiver) throws Exception {
74+
LOG.info("{}{}", prefix, element);
75+
receiver.output(element);
7476
}
7577
}
7678
}

0 commit comments

Comments
 (0)