-
Notifications
You must be signed in to change notification settings - Fork 13k
/
SlotPoolBatchSlotRequestTest.java
309 lines (251 loc) · 13.5 KB
/
SlotPoolBatchSlotRequestTest.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
307
308
309
/*
* 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.jobmaster.slotpool;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
import org.apache.flink.runtime.resourcemanager.exceptions.UnfulfillableSlotRequestException;
import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;
import org.apache.flink.util.clock.Clock;
import org.apache.flink.util.clock.ManualClock;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.fail;
/**
* Tests for batch slot requests.
*/
public class SlotPoolBatchSlotRequestTest extends TestLogger {
private static final ResourceProfile resourceProfile = ResourceProfile.fromResources(1.0, 1024);
private static final ResourceProfile smallerResourceProfile = ResourceProfile.fromResources(0.5, 512);
public static final CompletableFuture[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture[0];
private static ScheduledExecutorService singleThreadScheduledExecutorService;
private static ComponentMainThreadExecutor mainThreadExecutor;
@BeforeClass
public static void setupClass() {
singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService);
}
@AfterClass
public static void teardownClass() {
if (singleThreadScheduledExecutorService != null) {
singleThreadScheduledExecutorService.shutdownNow();
}
}
/**
* Tests that a batch slot request fails if there is no slot which can fulfill the
* slot request.
*/
@Test
public void testPendingBatchSlotRequestTimeout() throws Exception {
try (final SlotPoolImpl slotPool = createAndSetUpSlotPool(mainThreadExecutor, null, Time.milliseconds(2L))) {
final CompletableFuture<PhysicalSlot> slotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(
slotPool,
mainThreadExecutor,
ResourceProfile.UNKNOWN);
try {
slotFuture.get();
fail("Expected that slot future times out.");
} catch (ExecutionException ee) {
assertThat(ExceptionUtils.stripExecutionException(ee), instanceOf(TimeoutException.class));
}
}
}
/**
* Tests that a batch slot request won't time out if there exists a slot in the
* SlotPool which fulfills the requested {@link ResourceProfile}.
*/
@Test
public void testPendingBatchSlotRequestDoesNotTimeoutIfFulfillingSlotExists() throws Exception {
final Time batchSlotTimeout = Time.milliseconds(2L);
final ComponentMainThreadExecutor directMainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
final ManualClock clock = new ManualClock();
try (final TestingSlotPoolImpl slotPool = createAndSetUpSlotPool(
directMainThreadExecutor,
null,
batchSlotTimeout,
clock)) {
SlotPoolUtils.offerSlots(slotPool, directMainThreadExecutor, Collections.singletonList(resourceProfile));
final CompletableFuture<PhysicalSlot> firstSlotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, resourceProfile);
final CompletableFuture<PhysicalSlot> secondSlotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, ResourceProfile.UNKNOWN);
final CompletableFuture<PhysicalSlot> thirdSlotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, smallerResourceProfile);
final List<CompletableFuture<PhysicalSlot>> slotFutures = Arrays.asList(firstSlotFuture, secondSlotFuture, thirdSlotFuture);
advanceTimeAndTriggerCheckBatchSlotTimeout(slotPool, clock, batchSlotTimeout);
for (CompletableFuture<PhysicalSlot> slotFuture : slotFutures) {
assertThat(slotFuture.isDone(), is(false));
}
}
}
/**
* Tests that a batch slot request does not react to {@link SlotPool#failAllocation(AllocationID, Exception)}
* signals whose exception is not {@link UnfulfillableSlotRequestException}.
*/
@Test
public void testPendingBatchSlotRequestDoesNotFailIfAllocationFails() throws Exception {
final TestingResourceManagerGateway testingResourceManagerGateway = new TestingResourceManagerGateway();
final CompletableFuture<AllocationID> allocationIdFuture = new CompletableFuture<>();
testingResourceManagerGateway.setRequestSlotConsumer(slotRequest -> allocationIdFuture.complete(slotRequest.getAllocationId()));
final ComponentMainThreadExecutor directMainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
final Time batchSlotTimeout = Time.milliseconds(1000L);
try (final SlotPoolImpl slotPool = createAndSetUpSlotPool(
directMainThreadExecutor,
testingResourceManagerGateway,
batchSlotTimeout)) {
final CompletableFuture<PhysicalSlot> slotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, resourceProfile);
SlotPoolUtils.failAllocation(slotPool, directMainThreadExecutor, allocationIdFuture.get(), new FlinkException("Failed request"));
assertThat(slotFuture.isDone(), is(false));
}
}
/**
* Tests that a batch slot request does react to {@link SlotPool#failAllocation(AllocationID, Exception)}
* signals whose exception is {@link UnfulfillableSlotRequestException}.
*/
@Test
public void testPendingBatchSlotRequestFailsIfAllocationFailsUnfulfillably() throws Exception {
final TestingResourceManagerGateway testingResourceManagerGateway = new TestingResourceManagerGateway();
final CompletableFuture<AllocationID> allocationIdFuture = new CompletableFuture<>();
testingResourceManagerGateway.setRequestSlotConsumer(slotRequest -> allocationIdFuture.complete(slotRequest.getAllocationId()));
final ComponentMainThreadExecutor directMainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
try (final SlotPoolImpl slotPool = new SlotPoolBuilder(directMainThreadExecutor)
.setResourceManagerGateway(testingResourceManagerGateway)
.build()) {
final CompletableFuture<PhysicalSlot> slotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, resourceProfile);
SlotPoolUtils.failAllocation(slotPool, directMainThreadExecutor, allocationIdFuture.get(),
new UnfulfillableSlotRequestException(new AllocationID(), ResourceProfile.UNKNOWN));
assertThat(slotFuture.isCompletedExceptionally(), is(true));
}
}
/**
* Tests that a batch slot request won't fail if its resource manager request fails with exceptions other than
* {@link UnfulfillableSlotRequestException}.
*/
@Test
public void testPendingBatchSlotRequestDoesNotFailIfRMRequestFails() throws Exception {
final TestingResourceManagerGateway testingResourceManagerGateway = new TestingResourceManagerGateway();
testingResourceManagerGateway.setRequestSlotFuture(FutureUtils.completedExceptionally(new FlinkException("Failed request")));
final ComponentMainThreadExecutor directMainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
final Time batchSlotTimeout = Time.milliseconds(1000L);
try (final SlotPoolImpl slotPool = createAndSetUpSlotPool(
directMainThreadExecutor,
testingResourceManagerGateway,
batchSlotTimeout)) {
final CompletableFuture<PhysicalSlot> slotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, resourceProfile);
assertThat(slotFuture.isDone(), is(false));
}
}
/**
* Tests that a batch slot request fails if its resource manager request fails with {@link UnfulfillableSlotRequestException}.
*/
@Test
public void testPendingBatchSlotRequestFailsIfRMRequestFailsUnfulfillably() throws Exception {
final TestingResourceManagerGateway testingResourceManagerGateway = new TestingResourceManagerGateway();
testingResourceManagerGateway.setRequestSlotFuture(FutureUtils.completedExceptionally(
new UnfulfillableSlotRequestException(new AllocationID(), ResourceProfile.UNKNOWN)));
final ComponentMainThreadExecutor directMainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
try (final SlotPoolImpl slotPool = new SlotPoolBuilder(directMainThreadExecutor)
.setResourceManagerGateway(testingResourceManagerGateway)
.build()) {
final CompletableFuture<PhysicalSlot> slotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, resourceProfile);
assertThat(slotFuture.isCompletedExceptionally(), is(true));
}
}
/**
* Tests that a pending batch slot request times out after the last fulfilling slot gets
* released.
*/
@Test
public void testPendingBatchSlotRequestTimeoutAfterSlotRelease() throws Exception {
final ComponentMainThreadExecutor directMainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
final ManualClock clock = new ManualClock();
final Time batchSlotTimeout = Time.milliseconds(1000L);
try (final TestingSlotPoolImpl slotPool = createAndSetUpSlotPool(
directMainThreadExecutor,
null,
batchSlotTimeout,
clock)) {
final ResourceID taskManagerResourceId = SlotPoolUtils.offerSlots(slotPool, directMainThreadExecutor, Collections.singletonList(resourceProfile));
final CompletableFuture<PhysicalSlot> firstSlotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, resourceProfile);
final CompletableFuture<PhysicalSlot> secondSlotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, ResourceProfile.UNKNOWN);
final CompletableFuture<PhysicalSlot> thirdSlotFuture = SlotPoolUtils.requestNewAllocatedBatchSlot(slotPool, directMainThreadExecutor, smallerResourceProfile);
final List<CompletableFuture<PhysicalSlot>> slotFutures = Arrays.asList(firstSlotFuture, secondSlotFuture, thirdSlotFuture);
// initial batch slot timeout check
advanceTimeAndTriggerCheckBatchSlotTimeout(slotPool, clock, batchSlotTimeout);
assertThat(CompletableFuture.anyOf(slotFutures.toArray(COMPLETABLE_FUTURES_EMPTY_ARRAY)).isDone(), is(false));
SlotPoolUtils.releaseTaskManager(slotPool, directMainThreadExecutor, taskManagerResourceId);
advanceTimeAndTriggerCheckBatchSlotTimeout(slotPool, clock, batchSlotTimeout);
for (CompletableFuture<PhysicalSlot> slotFuture : slotFutures) {
assertThat(slotFuture.isCompletedExceptionally(), is(true));
try {
slotFuture.get();
fail("Expected that the slot future times out.");
} catch (ExecutionException ee) {
assertThat(ExceptionUtils.stripExecutionException(ee), instanceOf(TimeoutException.class));
}
}
}
}
private void advanceTimeAndTriggerCheckBatchSlotTimeout(TestingSlotPoolImpl slotPool, ManualClock clock, Time batchSlotTimeout) {
// trigger batch slot timeout check which marks unfulfillable slots
slotPool.triggerCheckBatchSlotTimeout();
// advance clock behind timeout
clock.advanceTime(batchSlotTimeout.toMilliseconds() + 1L, TimeUnit.MILLISECONDS);
// timeout all as unfulfillable marked slots
slotPool.triggerCheckBatchSlotTimeout();
}
private TestingSlotPoolImpl createAndSetUpSlotPool(
final ComponentMainThreadExecutor componentMainThreadExecutor,
@Nullable final ResourceManagerGateway resourceManagerGateway,
final Time batchSlotTimeout) throws Exception {
return new SlotPoolBuilder(componentMainThreadExecutor)
.setResourceManagerGateway(resourceManagerGateway)
.setBatchSlotTimeout(batchSlotTimeout)
.build();
}
private TestingSlotPoolImpl createAndSetUpSlotPool(
final ComponentMainThreadExecutor componentMainThreadExecutor,
@Nullable final ResourceManagerGateway resourceManagerGateway,
final Time batchSlotTimeout,
final Clock clock) throws Exception {
return new SlotPoolBuilder(componentMainThreadExecutor)
.setResourceManagerGateway(resourceManagerGateway)
.setBatchSlotTimeout(batchSlotTimeout)
.setClock(clock)
.build();
}
}