forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 2
/
KeyedStateBootstrapOperator.java
115 lines (93 loc) · 3.96 KB
/
KeyedStateBootstrapOperator.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
112
113
114
/*
* 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.connectors.savepoint.operators;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connectors.savepoint.functions.KeyedStateBootstrapFunction;
import org.apache.flink.connectors.savepoint.output.BoundedOperator;
import org.apache.flink.connectors.savepoint.output.SnapshotUtils;
import org.apache.flink.connectors.savepoint.output.TaggedOperatorSubtaskState;
import org.apache.flink.connectors.savepoint.runtime.VoidTriggerable;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.VoidNamespace;
import org.apache.flink.runtime.state.VoidNamespaceSerializer;
import org.apache.flink.streaming.api.SimpleTimerService;
import org.apache.flink.streaming.api.TimerService;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.Preconditions;
/**
* A {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing {@link
* KeyedStateBootstrapFunction}'s.
*/
@Internal
public class KeyedStateBootstrapOperator<K, IN>
extends AbstractUdfStreamOperator<TaggedOperatorSubtaskState, KeyedStateBootstrapFunction<K, IN>>
implements OneInputStreamOperator<IN, TaggedOperatorSubtaskState>,
BoundedOperator {
private static final long serialVersionUID = 1L;
private final long timestamp;
private final Path savepointPath;
private transient KeyedStateBootstrapOperator<K, IN>.ContextImpl context;
public KeyedStateBootstrapOperator(long timestamp, Path savepointPath, KeyedStateBootstrapFunction<K, IN> function) {
super(function);
this.timestamp = timestamp;
this.savepointPath = savepointPath;
}
@Override
public void open() throws Exception {
super.open();
InternalTimerService<VoidNamespace> internalTimerService = getInternalTimerService(
"user-timers",
VoidNamespaceSerializer.INSTANCE,
VoidTriggerable.instance());
TimerService timerService = new SimpleTimerService(internalTimerService);
context = new KeyedStateBootstrapOperator<K, IN>.ContextImpl(userFunction, timerService);
}
@Override
public void processElement(StreamRecord<IN> element) throws Exception {
userFunction.processElement(element.getValue(), context);
}
@Override
public void endInput() throws Exception {
TaggedOperatorSubtaskState state = SnapshotUtils.snapshot(
this,
getRuntimeContext().getIndexOfThisSubtask(),
timestamp,
getContainingTask().getCheckpointStorage(),
savepointPath);
output.collect(new StreamRecord<>(state));
}
private class ContextImpl extends KeyedStateBootstrapFunction<K, IN>.Context {
private final TimerService timerService;
ContextImpl(KeyedStateBootstrapFunction<K, IN> function, TimerService timerService) {
function.super();
this.timerService = Preconditions.checkNotNull(timerService);
}
@Override
public TimerService timerService() {
return timerService;
}
@Override
@SuppressWarnings("unchecked")
public K getCurrentKey() {
return (K) KeyedStateBootstrapOperator.this.getCurrentKey();
}
}
}