/
StreamGraphHasherV2.java
306 lines (266 loc) · 11.9 KB
/
StreamGraphHasherV2.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
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
/*
* 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.streaming.api.graph;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.streaming.api.operators.UdfStreamOperatorFactory;
import org.apache.flink.shaded.guava18.com.google.common.hash.HashFunction;
import org.apache.flink.shaded.guava18.com.google.common.hash.Hasher;
import org.apache.flink.shaded.guava18.com.google.common.hash.Hashing;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.charset.Charset;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import static org.apache.flink.util.StringUtils.byteToHexString;
/**
* StreamGraphHasher from Flink 1.2. This contains duplicated code to ensure that the algorithm does
* not change with future Flink versions.
*
* <p>DO NOT MODIFY THIS CLASS
*/
public class StreamGraphHasherV2 implements StreamGraphHasher {
private static final Logger LOG = LoggerFactory.getLogger(StreamGraphHasherV2.class);
/**
* Returns a map with a hash for each {@link StreamNode} of the {@link StreamGraph}. The hash is
* used as the {@link JobVertexID} in order to identify nodes across job submissions if they
* didn't change.
*
* <p>The complete {@link StreamGraph} is traversed. The hash is either computed from the
* transformation's user-specified id (see {@link Transformation#getUid()}) or generated in a
* deterministic way.
*
* <p>The generated hash is deterministic with respect to:
*
* <ul>
* <li>node-local properties (node ID),
* <li>chained output nodes, and
* <li>input nodes hashes
* </ul>
*
* @return A map from {@link StreamNode#id} to hash as 16-byte array.
*/
@Override
public Map<Integer, byte[]> traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) {
// The hash function used to generate the hash
final HashFunction hashFunction = Hashing.murmur3_128(0);
final Map<Integer, byte[]> hashes = new HashMap<>();
Set<Integer> visited = new HashSet<>();
Queue<StreamNode> remaining = new ArrayDeque<>();
// We need to make the source order deterministic. The source IDs are
// not returned in the same order, which means that submitting the same
// program twice might result in different traversal, which breaks the
// deterministic hash assignment.
List<Integer> sources = new ArrayList<>();
for (Integer sourceNodeId : streamGraph.getSourceIDs()) {
sources.add(sourceNodeId);
}
Collections.sort(sources);
//
// Traverse the graph in a breadth-first manner. Keep in mind that
// the graph is not a tree and multiple paths to nodes can exist.
//
// Start with source nodes
for (Integer sourceNodeId : sources) {
remaining.add(streamGraph.getStreamNode(sourceNodeId));
visited.add(sourceNodeId);
}
StreamNode currentNode;
while ((currentNode = remaining.poll()) != null) {
// Generate the hash code. Because multiple path exist to each
// node, we might not have all required inputs available to
// generate the hash code.
if (generateNodeHash(
currentNode,
hashFunction,
hashes,
streamGraph.isChainingEnabled(),
streamGraph)) {
// Add the child nodes
for (StreamEdge outEdge : currentNode.getOutEdges()) {
StreamNode child = streamGraph.getTargetVertex(outEdge);
if (!visited.contains(child.getId())) {
remaining.add(child);
visited.add(child.getId());
}
}
} else {
// We will revisit this later.
visited.remove(currentNode.getId());
}
}
return hashes;
}
/**
* Generates a hash for the node and returns whether the operation was successful.
*
* @param node The node to generate the hash for
* @param hashFunction The hash function to use
* @param hashes The current state of generated hashes
* @return <code>true</code> if the node hash has been generated. <code>false</code>, otherwise.
* If the operation is not successful, the hash needs be generated at a later point when all
* input is available.
* @throws IllegalStateException If node has user-specified hash and is intermediate node of a
* chain
*/
private boolean generateNodeHash(
StreamNode node,
HashFunction hashFunction,
Map<Integer, byte[]> hashes,
boolean isChainingEnabled,
StreamGraph streamGraph) {
// Check for user-specified ID
String userSpecifiedHash = node.getTransformationUID();
if (userSpecifiedHash == null) {
// Check that all input nodes have their hashes computed
for (StreamEdge inEdge : node.getInEdges()) {
// If the input node has not been visited yet, the current
// node will be visited again at a later point when all input
// nodes have been visited and their hashes set.
if (!hashes.containsKey(inEdge.getSourceId())) {
return false;
}
}
Hasher hasher = hashFunction.newHasher();
byte[] hash =
generateDeterministicHash(node, hasher, hashes, isChainingEnabled, streamGraph);
if (hashes.put(node.getId(), hash) != null) {
// Sanity check
throw new IllegalStateException(
"Unexpected state. Tried to add node hash "
+ "twice. This is probably a bug in the JobGraph generator.");
}
return true;
} else {
Hasher hasher = hashFunction.newHasher();
byte[] hash = generateUserSpecifiedHash(node, hasher);
for (byte[] previousHash : hashes.values()) {
if (Arrays.equals(previousHash, hash)) {
throw new IllegalArgumentException(
"Hash collision on user-specified ID "
+ "\""
+ userSpecifiedHash
+ "\". "
+ "Most likely cause is a non-unique ID. Please check that all IDs "
+ "specified via `uid(String)` are unique.");
}
}
if (hashes.put(node.getId(), hash) != null) {
// Sanity check
throw new IllegalStateException(
"Unexpected state. Tried to add node hash "
+ "twice. This is probably a bug in the JobGraph generator.");
}
return true;
}
}
/** Generates a hash from a user-specified ID. */
private byte[] generateUserSpecifiedHash(StreamNode node, Hasher hasher) {
hasher.putString(node.getTransformationUID(), Charset.forName("UTF-8"));
return hasher.hash().asBytes();
}
/** Generates a deterministic hash from node-local properties and input and output edges. */
private byte[] generateDeterministicHash(
StreamNode node,
Hasher hasher,
Map<Integer, byte[]> hashes,
boolean isChainingEnabled,
StreamGraph streamGraph) {
// Include stream node to hash. We use the current size of the computed
// hashes as the ID. We cannot use the node's ID, because it is
// assigned from a static counter. This will result in two identical
// programs having different hashes.
generateNodeLocalHash(hasher, hashes.size());
// Include chained nodes to hash
for (StreamEdge outEdge : node.getOutEdges()) {
if (isChainable(outEdge, isChainingEnabled, streamGraph)) {
// Use the hash size again, because the nodes are chained to
// this node. This does not add a hash for the chained nodes.
generateNodeLocalHash(hasher, hashes.size());
}
}
byte[] hash = hasher.hash().asBytes();
// Make sure that all input nodes have their hash set before entering
// this loop (calling this method).
for (StreamEdge inEdge : node.getInEdges()) {
byte[] otherHash = hashes.get(inEdge.getSourceId());
// Sanity check
if (otherHash == null) {
throw new IllegalStateException(
"Missing hash for input node "
+ streamGraph.getSourceVertex(inEdge)
+ ". Cannot generate hash for "
+ node
+ ".");
}
for (int j = 0; j < hash.length; j++) {
hash[j] = (byte) (hash[j] * 37 ^ otherHash[j]);
}
}
if (LOG.isDebugEnabled()) {
String udfClassName = "";
if (node.getOperatorFactory() instanceof UdfStreamOperatorFactory) {
udfClassName =
((UdfStreamOperatorFactory) node.getOperatorFactory())
.getUserFunctionClassName();
}
LOG.debug(
"Generated hash '"
+ byteToHexString(hash)
+ "' for node "
+ "'"
+ node.toString()
+ "' {id: "
+ node.getId()
+ ", "
+ "parallelism: "
+ node.getParallelism()
+ ", "
+ "user function: "
+ udfClassName
+ "}");
}
return hash;
}
/**
* Applies the {@link Hasher} to the {@link StreamNode} . The hasher encapsulates the current
* state of the hash.
*
* <p>The specified ID is local to this node. We cannot use the {@link StreamNode#id}, because
* it is incremented in a static counter. Therefore, the IDs for identical jobs will otherwise
* be different.
*/
private void generateNodeLocalHash(Hasher hasher, int id) {
// This resolves conflicts for otherwise identical source nodes. BUT
// the generated hash codes depend on the ordering of the nodes in the
// stream graph.
hasher.putInt(id);
}
private boolean isChainable(
StreamEdge edge, boolean isChainingEnabled, StreamGraph streamGraph) {
return isChainingEnabled && StreamingJobGraphGenerator.isChainable(edge, streamGraph);
}
}