-
Notifications
You must be signed in to change notification settings - Fork 13k
/
StateReaderOperator.java
151 lines (121 loc) · 5.31 KB
/
StateReaderOperator.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
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
/*
* 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.state.api.input.operator;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.functions.DefaultOpenContext;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.SerializerFactory;
import org.apache.flink.api.common.functions.util.FunctionUtils;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.runtime.state.KeyedStateBackend;
import org.apache.flink.state.api.runtime.SavepointRuntimeContext;
import org.apache.flink.state.api.runtime.VoidTriggerable;
import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.operators.KeyContext;
import org.apache.flink.util.CloseableIterator;
import org.apache.flink.util.Collector;
import org.apache.flink.util.Preconditions;
import java.io.Serializable;
/**
* Base class for executing functions that read keyed state.
*
* @param <F> The type of the user function.
* @param <KEY> The key type.
* @param <N> The namespace type.
* @param <OUT> The output type.
*/
@Internal
public abstract class StateReaderOperator<F extends Function, KEY, N, OUT>
implements KeyContext, AutoCloseable, Serializable {
private static final long serialVersionUID = 1L;
protected final F function;
private final TypeInformation<KEY> keyType;
protected final TypeSerializer<N> namespaceSerializer;
private transient SerializerFactory serializerFactory;
private transient KeyedStateBackend<KEY> keyedStateBackend;
private transient TypeSerializer<KEY> keySerializer;
private transient InternalTimeServiceManager<KEY> timerServiceManager;
protected StateReaderOperator(
F function, TypeInformation<KEY> keyType, TypeSerializer<N> namespaceSerializer) {
Preconditions.checkNotNull(function, "The user function must not be null");
Preconditions.checkNotNull(keyType, "The key type must not be null");
Preconditions.checkNotNull(
namespaceSerializer, "The namespace serializer must not be null");
this.function = function;
this.keyType = keyType;
this.namespaceSerializer = namespaceSerializer;
}
public abstract void processElement(KEY key, N namespace, Collector<OUT> out) throws Exception;
public abstract CloseableIterator<Tuple2<KEY, N>> getKeysAndNamespaces(
SavepointRuntimeContext ctx) throws Exception;
public final void setup(
SerializerFactory serializerFactory,
KeyedStateBackend<KEY> keyKeyedStateBackend,
InternalTimeServiceManager<KEY> timerServiceManager,
SavepointRuntimeContext ctx) {
this.serializerFactory = serializerFactory;
this.keyedStateBackend = keyKeyedStateBackend;
this.timerServiceManager = timerServiceManager;
this.keySerializer = serializerFactory.createSerializer(keyType);
FunctionUtils.setFunctionRuntimeContext(function, ctx);
}
protected final InternalTimerService<N> getInternalTimerService(String name) {
return timerServiceManager.getInternalTimerService(
name, keySerializer, namespaceSerializer, VoidTriggerable.instance());
}
public void open() throws Exception {
FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE);
}
public void close() throws Exception {
Exception exception = null;
try {
FunctionUtils.closeFunction(function);
} catch (Exception e) {
// The state backend must always be closed
// to release native resources.
exception = e;
}
if (keyedStateBackend != null) {
keyedStateBackend.dispose();
}
if (exception != null) {
throw exception;
}
}
@Override
@SuppressWarnings("unchecked")
public final void setCurrentKey(Object key) {
keyedStateBackend.setCurrentKey((KEY) key);
}
@Override
public final Object getCurrentKey() {
return keyedStateBackend.getCurrentKey();
}
public final KeyedStateBackend<KEY> getKeyedStateBackend() {
return keyedStateBackend;
}
public final TypeInformation<KEY> getKeyType() {
return keyType;
}
public final SerializerFactory getSerializerFactory() {
return this.serializerFactory;
}
}