/
RuntimeUDFContext.java
115 lines (97 loc) · 4.1 KB
/
RuntimeUDFContext.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
/*
* 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.api.common.functions.util;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Future;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.core.fs.Path;
/**
* A standalone implementation of the {@link RuntimeContext}, created by runtime UDF operators.
*/
public class RuntimeUDFContext extends AbstractRuntimeUDFContext {
private final HashMap<String, Object> initializedBroadcastVars = new HashMap<String, Object>();
private final HashMap<String, List<?>> uninitializedBroadcastVars = new HashMap<String, List<?>>();
public RuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig) {
super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig);
}
public RuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, Map<String, Future<Path>> cpTasks) {
super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig, cpTasks);
}
@Override
@SuppressWarnings("unchecked")
public <RT> List<RT> getBroadcastVariable(String name) {
// check if we have an initialized version
Object o = this.initializedBroadcastVars.get(name);
if (o != null) {
if (o instanceof List) {
return (List<RT>) o;
}
else {
throw new IllegalStateException("The broadcast variable with name '" + name +
"' is not a List. A different call must have requested this variable with a BroadcastVariableInitializer.");
}
}
else {
List<?> uninitialized = this.uninitializedBroadcastVars.remove(name);
if (uninitialized != null) {
this.initializedBroadcastVars.put(name, uninitialized);
return (List<RT>) uninitialized;
}
else {
throw new IllegalArgumentException("The broadcast variable with name '" + name + "' has not been set.");
}
}
}
@SuppressWarnings("unchecked")
@Override
public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
// check if we have an initialized version
Object o = this.initializedBroadcastVars.get(name);
if (o != null) {
return (C) o;
}
else {
List<T> uninitialized = (List<T>) this.uninitializedBroadcastVars.remove(name);
if (uninitialized != null) {
C result = initializer.initializeBroadcastVariable(uninitialized);
this.initializedBroadcastVars.put(name, result);
return result;
}
else {
throw new IllegalArgumentException("The broadcast variable with name '" + name + "' has not been set.");
}
}
}
// --------------------------------------------------------------------------------------------
public void setBroadcastVariable(String name, List<?> value) {
this.uninitializedBroadcastVars.put(name, value);
this.initializedBroadcastVars.remove(name);
}
public void clearBroadcastVariable(String name) {
this.uninitializedBroadcastVars.remove(name);
this.initializedBroadcastVars.remove(name);
}
public void clearAllBroadcastVariables() {
this.uninitializedBroadcastVars.clear();
this.initializedBroadcastVars.clear();
}
}