-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
GroupAlsoByWindowViaWindowSetDoFn.java
111 lines (99 loc) · 4.69 KB
/
GroupAlsoByWindowViaWindowSetDoFn.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
/*
* 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.beam.runners.core;
import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor;
import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
import org.apache.beam.runners.core.triggers.TriggerStateMachines;
import org.apache.beam.sdk.transforms.Aggregator;
import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.SystemDoFnInternal;
import org.apache.beam.sdk.util.TimerInternals;
import org.apache.beam.sdk.util.WindowingStrategy;
import org.apache.beam.sdk.util.state.StateInternals;
import org.apache.beam.sdk.util.state.StateInternalsFactory;
import org.apache.beam.sdk.values.KV;
/**
* A general {@link GroupAlsoByWindowsDoFn}. This delegates all of the logic to the
* {@link ReduceFnRunner}.
*/
@SystemDoFnInternal
public class GroupAlsoByWindowViaWindowSetDoFn<
K, InputT, OutputT, W extends BoundedWindow, RinT extends KeyedWorkItem<K, InputT>>
extends OldDoFn<RinT, KV<K, OutputT>> implements ReduceFnExecutor<K, InputT, OutputT, W> {
public static <K, InputT, OutputT, W extends BoundedWindow>
OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> create(
WindowingStrategy<?, W> strategy,
StateInternalsFactory<K> stateInternalsFactory,
SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn) {
return new GroupAlsoByWindowViaWindowSetDoFn<>(strategy, stateInternalsFactory, reduceFn);
}
protected final Aggregator<Long, Long> droppedDueToClosedWindow =
createAggregator(
GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, Sum.ofLongs());
protected final Aggregator<Long, Long> droppedDueToLateness =
createAggregator(GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_LATENESS_COUNTER, Sum.ofLongs());
private final WindowingStrategy<Object, W> windowingStrategy;
private final StateInternalsFactory<K> stateInternalsFactory;
private SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn;
private GroupAlsoByWindowViaWindowSetDoFn(
WindowingStrategy<?, W> windowingStrategy,
StateInternalsFactory<K> stateInternalsFactory,
SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn) {
@SuppressWarnings("unchecked")
WindowingStrategy<Object, W> noWildcard = (WindowingStrategy<Object, W>) windowingStrategy;
this.windowingStrategy = noWildcard;
this.reduceFn = reduceFn;
this.stateInternalsFactory = stateInternalsFactory;
}
@Override
public void processElement(ProcessContext c) throws Exception {
KeyedWorkItem<K, InputT> keyedWorkItem = c.element();
K key = keyedWorkItem.key();
TimerInternals timerInternals = c.windowingInternals().timerInternals();
StateInternals<K> stateInternals = stateInternalsFactory.stateInternalsForKey(key);
ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
new ReduceFnRunner<>(
key,
windowingStrategy,
ExecutableTriggerStateMachine.create(
TriggerStateMachines.stateMachineForTrigger(windowingStrategy.getTrigger())),
stateInternals,
timerInternals,
WindowingInternalsAdapters.outputWindowedValue(c.windowingInternals()),
WindowingInternalsAdapters.sideInputReader(c.windowingInternals()),
droppedDueToClosedWindow,
reduceFn,
c.getPipelineOptions());
reduceFnRunner.processElements(keyedWorkItem.elementsIterable());
reduceFnRunner.onTimers(keyedWorkItem.timersIterable());
reduceFnRunner.persist();
}
@Override
public OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> asDoFn() {
// Safe contravariant cast
@SuppressWarnings("unchecked")
OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> asFn =
(OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>>) this;
return asFn;
}
@Override
public Aggregator<Long, Long> getDroppedDueToLatenessAggregator() {
return droppedDueToLateness;
}
}