-
Notifications
You must be signed in to change notification settings - Fork 19
/
DefaultExecutionGraphCacheTest.java
298 lines (239 loc) · 12.8 KB
/
DefaultExecutionGraphCacheTest.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
/*
* 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.runtime.rest.handler.legacy;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder;
import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
import org.apache.flink.runtime.webmonitor.RestfulGateway;
import org.apache.flink.runtime.webmonitor.TestingRestfulGateway;
import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.TestLogger;
import org.apache.flink.util.concurrent.FutureUtils;
import org.hamcrest.Matchers;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/** Tests for the {@link DefaultExecutionGraphCache}. */
public class DefaultExecutionGraphCacheTest extends TestLogger {
private static ExecutionGraphInfo expectedExecutionGraphInfo;
private static final JobID expectedJobId = new JobID();
@BeforeClass
public static void setup() {
expectedExecutionGraphInfo =
new ExecutionGraphInfo(new ArchivedExecutionGraphBuilder().build());
}
/** Tests that we can cache AccessExecutionGraphs over multiple accesses. */
@Test
public void testExecutionGraphCaching() throws Exception {
final Time timeout = Time.milliseconds(100L);
final Time timeToLive = Time.hours(1L);
final CountingRestfulGateway restfulGateway =
createCountingRestfulGateway(
expectedJobId,
CompletableFuture.completedFuture(expectedExecutionGraphInfo));
try (ExecutionGraphCache executionGraphCache =
new DefaultExecutionGraphCache(timeout, timeToLive)) {
CompletableFuture<ExecutionGraphInfo> executionGraphInfoFuture =
executionGraphCache.getExecutionGraphInfo(expectedJobId, restfulGateway);
assertEquals(expectedExecutionGraphInfo, executionGraphInfoFuture.get());
executionGraphInfoFuture =
executionGraphCache.getExecutionGraphInfo(expectedJobId, restfulGateway);
assertEquals(expectedExecutionGraphInfo, executionGraphInfoFuture.get());
assertThat(restfulGateway.getNumRequestJobCalls(), Matchers.equalTo(1));
}
}
/** Tests that an AccessExecutionGraph is invalidated after its TTL expired. */
@Test
public void testExecutionGraphEntryInvalidation() throws Exception {
final Time timeout = Time.milliseconds(100L);
final Time timeToLive = Time.milliseconds(1L);
final CountingRestfulGateway restfulGateway =
createCountingRestfulGateway(
expectedJobId,
CompletableFuture.completedFuture(expectedExecutionGraphInfo),
CompletableFuture.completedFuture(expectedExecutionGraphInfo));
try (ExecutionGraphCache executionGraphCache =
new DefaultExecutionGraphCache(timeout, timeToLive)) {
CompletableFuture<ExecutionGraphInfo> executionGraphInfoFuture =
executionGraphCache.getExecutionGraphInfo(expectedJobId, restfulGateway);
assertEquals(expectedExecutionGraphInfo, executionGraphInfoFuture.get());
// sleep for the TTL
Thread.sleep(timeToLive.toMilliseconds() * 5L);
CompletableFuture<ExecutionGraphInfo> executionGraphInfoFuture2 =
executionGraphCache.getExecutionGraphInfo(expectedJobId, restfulGateway);
assertEquals(expectedExecutionGraphInfo, executionGraphInfoFuture2.get());
assertThat(restfulGateway.getNumRequestJobCalls(), Matchers.equalTo(2));
}
}
/**
* Tests that a failure in requesting an AccessExecutionGraph from the gateway, will not create
* a cache entry --> another cache request will trigger a new gateway request.
*/
@Test
public void testImmediateCacheInvalidationAfterFailure() throws Exception {
final Time timeout = Time.milliseconds(100L);
final Time timeToLive = Time.hours(1L);
// let's first answer with a JobNotFoundException and then only with the correct result
final CountingRestfulGateway restfulGateway =
createCountingRestfulGateway(
expectedJobId,
FutureUtils.completedExceptionally(
new FlinkJobNotFoundException(expectedJobId)),
CompletableFuture.completedFuture(expectedExecutionGraphInfo));
try (ExecutionGraphCache executionGraphCache =
new DefaultExecutionGraphCache(timeout, timeToLive)) {
CompletableFuture<ExecutionGraphInfo> executionGraphFuture =
executionGraphCache.getExecutionGraphInfo(expectedJobId, restfulGateway);
try {
executionGraphFuture.get();
fail("The execution graph future should have been completed exceptionally.");
} catch (ExecutionException ee) {
assertTrue(ee.getCause() instanceof FlinkException);
}
CompletableFuture<ExecutionGraphInfo> executionGraphFuture2 =
executionGraphCache.getExecutionGraphInfo(expectedJobId, restfulGateway);
assertEquals(expectedExecutionGraphInfo, executionGraphFuture2.get());
}
}
/**
* Tests that cache entries are cleaned up when their TTL has expired upon calling {@link
* DefaultExecutionGraphCache#cleanup()}.
*/
@Test
public void testCacheEntryCleanup() throws Exception {
final Time timeout = Time.milliseconds(100L);
final Time timeToLive = Time.milliseconds(1L);
final JobID expectedJobId2 = new JobID();
final ExecutionGraphInfo expectedExecutionGraphInfo2 =
new ExecutionGraphInfo(new ArchivedExecutionGraphBuilder().build());
final AtomicInteger requestJobCalls = new AtomicInteger(0);
final TestingRestfulGateway restfulGateway =
new TestingRestfulGateway.Builder()
.setRequestExecutionGraphInfoFunction(
jobId -> {
requestJobCalls.incrementAndGet();
if (jobId.equals(expectedJobId)) {
return CompletableFuture.completedFuture(
expectedExecutionGraphInfo);
} else if (jobId.equals(expectedJobId2)) {
return CompletableFuture.completedFuture(
expectedExecutionGraphInfo2);
} else {
throw new AssertionError("Invalid job id received.");
}
})
.build();
try (ExecutionGraphCache executionGraphCache =
new DefaultExecutionGraphCache(timeout, timeToLive)) {
CompletableFuture<ExecutionGraphInfo> executionGraph1Future =
executionGraphCache.getExecutionGraphInfo(expectedJobId, restfulGateway);
CompletableFuture<ExecutionGraphInfo> executionGraph2Future =
executionGraphCache.getExecutionGraphInfo(expectedJobId2, restfulGateway);
assertEquals(expectedExecutionGraphInfo, executionGraph1Future.get());
assertEquals(expectedExecutionGraphInfo2, executionGraph2Future.get());
assertThat(requestJobCalls.get(), Matchers.equalTo(2));
Thread.sleep(timeToLive.toMilliseconds());
executionGraphCache.cleanup();
assertTrue(executionGraphCache.size() == 0);
}
}
/** Tests that concurrent accesses only trigger a single AccessExecutionGraph request. */
@Test
public void testConcurrentAccess() throws Exception {
final Time timeout = Time.milliseconds(100L);
final Time timeToLive = Time.hours(1L);
final CountingRestfulGateway restfulGateway =
createCountingRestfulGateway(
expectedJobId,
CompletableFuture.completedFuture(expectedExecutionGraphInfo));
final int numConcurrentAccesses = 10;
final ArrayList<CompletableFuture<ExecutionGraphInfo>> executionGraphFutures =
new ArrayList<>(numConcurrentAccesses);
final ExecutorService executor =
java.util.concurrent.Executors.newFixedThreadPool(numConcurrentAccesses);
try (ExecutionGraphCache executionGraphCache =
new DefaultExecutionGraphCache(timeout, timeToLive)) {
for (int i = 0; i < numConcurrentAccesses; i++) {
CompletableFuture<ExecutionGraphInfo> executionGraphFuture =
CompletableFuture.supplyAsync(
() ->
executionGraphCache.getExecutionGraphInfo(
expectedJobId, restfulGateway),
executor)
.thenCompose(Function.identity());
executionGraphFutures.add(executionGraphFuture);
}
final CompletableFuture<Collection<ExecutionGraphInfo>> allExecutionGraphFutures =
FutureUtils.combineAll(executionGraphFutures);
Collection<ExecutionGraphInfo> allExecutionGraphs = allExecutionGraphFutures.get();
for (ExecutionGraphInfo executionGraph : allExecutionGraphs) {
assertEquals(expectedExecutionGraphInfo, executionGraph);
}
assertThat(restfulGateway.getNumRequestJobCalls(), Matchers.equalTo(1));
} finally {
ExecutorUtils.gracefulShutdown(5000L, TimeUnit.MILLISECONDS, executor);
}
}
private CountingRestfulGateway createCountingRestfulGateway(
JobID jobId, CompletableFuture<ExecutionGraphInfo>... accessExecutionGraphs) {
final ConcurrentLinkedQueue<CompletableFuture<ExecutionGraphInfo>> queue =
new ConcurrentLinkedQueue<>(Arrays.asList(accessExecutionGraphs));
return new CountingRestfulGateway(jobId, ignored -> queue.poll());
}
/**
* {@link RestfulGateway} implementation which counts the number of {@link #requestJob(JobID,
* Time)} calls.
*/
private static class CountingRestfulGateway extends TestingRestfulGateway {
private final JobID expectedJobId;
private AtomicInteger numRequestJobCalls = new AtomicInteger(0);
private CountingRestfulGateway(
JobID expectedJobId,
Function<JobID, CompletableFuture<ExecutionGraphInfo>> requestJobFunction) {
this.expectedJobId = Preconditions.checkNotNull(expectedJobId);
this.requestExecutionGraphInfoFunction = Preconditions.checkNotNull(requestJobFunction);
}
@Override
public CompletableFuture<ExecutionGraphInfo> requestExecutionGraphInfo(
JobID jobId, Time timeout) {
assertThat(jobId, Matchers.equalTo(expectedJobId));
numRequestJobCalls.incrementAndGet();
return super.requestExecutionGraphInfo(jobId, timeout);
}
public int getNumRequestJobCalls() {
return numRequestJobCalls.get();
}
}
}