-
Notifications
You must be signed in to change notification settings - Fork 13k
/
BatchFineGrainedRecoveryITCase.java
631 lines (542 loc) · 22 KB
/
BatchFineGrainedRecoveryITCase.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
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
/*
* 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.test.recovery;
import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.functions.RichMapPartitionFunction;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.executiongraph.failover.FailoverStrategyLoader;
import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.messages.webmonitor.JobIdsWithStatusOverview;
import org.apache.flink.runtime.messages.webmonitor.JobIdsWithStatusOverview.JobIdWithStatus;
import org.apache.flink.runtime.minicluster.RpcServiceSharing;
import org.apache.flink.runtime.minicluster.TestingMiniCluster;
import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration.Builder;
import org.apache.flink.runtime.rest.RestClient;
import org.apache.flink.runtime.rest.RestClientConfiguration;
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.JobIdsWithStatusesOverviewHeaders;
import org.apache.flink.runtime.rest.messages.JobMessageParameters;
import org.apache.flink.runtime.rest.messages.JobVertexDetailsHeaders;
import org.apache.flink.runtime.rest.messages.JobVertexDetailsInfo;
import org.apache.flink.runtime.rest.messages.JobVertexMessageParameters;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.runtime.rest.messages.MessageParameters;
import org.apache.flink.runtime.rest.messages.ResponseBody;
import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders;
import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo;
import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptDetailsInfo;
import org.apache.flink.runtime.util.ExecutorThreadFactory;
import org.apache.flink.test.util.TestEnvironment;
import org.apache.flink.util.Collector;
import org.apache.flink.util.ConfigurationException;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.TestLogger;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.concurrent.GuardedBy;
import java.io.IOException;
import java.io.Serializable;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.LongStream;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertThat;
/**
* IT case for fine-grained recovery of batch jobs.
*
* <p>The test activates the region fail-over strategy to restart only failed producers.
* The test job is a sequence of non-parallel mappers. Each mapper writes to a blocking partition which means
* the next mapper starts when the previous is done. The mappers are not chained into one task which makes them
* separate fail-over regions.
*
* <p>The test verifies the fine-grained recovery by including one failure after a random record for each failure
* strategy in all mappers and comparing expected number of each mapper restarts against the actual restarts.
* There are multiple failure strategies:
*
* <ul>
* <li> The {@link ExceptionFailureStrategy} throws an exception in the user function code.
* Since all mappers are connected via blocking partitions, which should be re-used on failure, and the consumer
* of the mapper wasn't deployed yet, as the consumed partition was not fully produced yet, only the failed mapper
* should actually restart.
* <li> The {@link TaskExecutorFailureStrategy} abruptly shuts down the task executor. This leads to the loss
* of all previously completed and the in-progress mapper result partitions. The fail-over strategy should restart
* the current in-progress mapper which will get the {@link PartitionNotFoundException} because the previous result
* becomes unavailable and the previous mapper has to be restarted as well. The same should happen subsequently with
* all previous mappers. When the source is recomputed, all mappers has to be restarted again to recalculate their
* lost results.
* </ul>
*/
@RunWith(Parameterized.class)
public class BatchFineGrainedRecoveryITCase extends TestLogger {
private static final Logger LOG = LoggerFactory.getLogger(BatchFineGrainedRecoveryITCase.class);
private static final int EMITTED_RECORD_NUMBER = 1000;
private static final int MAP_NUMBER = 3;
private static final String MAP_PARTITION_TEST_PARTITION_MAPPER = "MapPartition (Test partition mapper ";
private static final Pattern MAPPER_NUMBER_IN_TASK_NAME_PATTERN =
Pattern.compile("MapPartition \\(Test partition mapper (\\d+)\\)");
/**
* Number of job failures for all mappers due to backtracking when the produced partitions get lost.
*
* <p>Each mapper failure produces number of backtracking failures (partition not found) which is the mapper index + 1,
* because all previous producers have to be restarted and they firstly will not find the previous result.
*/
private static final int ALL_MAPPERS_BACKTRACK_FAILURES = IntStream.range(0, MAP_NUMBER + 1).sum();
/**
* Max number of job failures.
*
* <p>For each possible mapper failure, it is all possible backtracking failures plus the generated failures themselves of each type.
*/
private static final int MAX_JOB_RESTART_ATTEMPTS = ALL_MAPPERS_BACKTRACK_FAILURES + 2 * MAP_NUMBER;
/**
* Expected attempt number for each mapper.
*/
private static final int[] EXPECTED_MAP_ATTEMPT_NUMBERS = IntStream
.range(0, MAP_NUMBER)
.map(i ->
// exception failure:
1 + // this mapper
i + // previous mappers
// TM failure:
(MAP_NUMBER - i - 1) + // subsequent mappers after PartitionNotFoundException
1 + // this mapper
1 + // this mapper after PartitionNotFoundException
i + // previous mappers
i) // previous mappers after PartitionNotFoundException
.toArray();
private static final String TASK_NAME_PREFIX = "Test partition mapper ";
private static final List<Long> EXPECTED_JOB_OUTPUT =
LongStream.range(MAP_NUMBER, EMITTED_RECORD_NUMBER + MAP_NUMBER).boxed().collect(Collectors.toList());
private static TestingMiniCluster miniCluster;
private static MiniClusterClient client;
private static AtomicInteger lastTaskManagerIndexInMiniCluster;
private static final Random rnd = new Random();
private static GlobalMapFailureTracker failureTracker;
@Parameterized.Parameter
public String failoverStrategyName;
@Parameterized.Parameters(name = "[{index}] failover strategy: {0}")
public static Object[] failoverStrategies() {
return new Object[] {
FailoverStrategyLoader.PIPELINED_REGION_RESTART_STRATEGY_NAME,
FailoverStrategyLoader.FAST_PIPELINED_REGION_RESTART_STRATEGY_NAME
};
}
@SuppressWarnings("OverlyBroadThrowsClause")
@Before
public void setup() throws Exception {
Configuration configuration = new Configuration();
configuration.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, failoverStrategyName);
miniCluster = new TestingMiniCluster(
new Builder()
.setNumTaskManagers(1)
.setNumSlotsPerTaskManager(1)
.setConfiguration(configuration)
.setRpcServiceSharing(RpcServiceSharing.DEDICATED)
.build(),
null);
miniCluster.start();
client = new MiniClusterClient(miniCluster);
lastTaskManagerIndexInMiniCluster = new AtomicInteger(0);
failureTracker = new GlobalMapFailureTracker(MAP_NUMBER);
}
@After
public void teardown() throws Exception {
if (miniCluster != null) {
miniCluster.close();
}
if (client != null) {
client.close();
}
}
@Test
public void testProgram() throws Exception {
ExecutionEnvironment env = createExecutionEnvironment();
DataSet<Long> input = env.generateSequence(0, EMITTED_RECORD_NUMBER - 1);
for (int trackingIndex = 0; trackingIndex < MAP_NUMBER; trackingIndex++) {
input = input
.mapPartition(new TestPartitionMapper(trackingIndex, createFailureStrategy(trackingIndex)))
.name(TASK_NAME_PREFIX + trackingIndex);
}
assertThat(input.collect(), is(EXPECTED_JOB_OUTPUT));
failureTracker.verify(getMapperAttempts());
}
private static FailureStrategy createFailureStrategy(int trackingIndex) {
int failWithExceptionAfterNumberOfProcessedRecords = rnd.nextInt(EMITTED_RECORD_NUMBER) + 1;
int failTaskExecutorAfterNumberOfProcessedRecords = rnd.nextInt(EMITTED_RECORD_NUMBER) + 1;
// it has to fail only once during one mapper run so that different failure strategies do not mess up each other stats
FailureStrategy failureStrategy = new OneTimeFailureStrategy(
new JoinedFailureStrategy(
new GloballyTrackingFailureStrategy(
new ExceptionFailureStrategy(failWithExceptionAfterNumberOfProcessedRecords)),
new GloballyTrackingFailureStrategy(
new TaskExecutorFailureStrategy(failTaskExecutorAfterNumberOfProcessedRecords))));
LOG.info("FailureStrategy for the mapper {}: {}", trackingIndex, failureStrategy);
return failureStrategy;
}
private static ExecutionEnvironment createExecutionEnvironment() {
@SuppressWarnings("StaticVariableUsedBeforeInitialization")
ExecutionEnvironment env = new TestEnvironment(miniCluster, 1, true);
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(MAX_JOB_RESTART_ATTEMPTS, Time.milliseconds(10)));
env.getConfig().setExecutionMode(ExecutionMode.BATCH_FORCED); // forces all partitions to be blocking
return env;
}
@SuppressWarnings({"StaticVariableUsedBeforeInitialization", "OverlyBroadThrowsClause"})
private static void restartTaskManager() throws Exception {
int tmi = lastTaskManagerIndexInMiniCluster.getAndIncrement();
try {
miniCluster.terminateTaskExecutor(tmi).get();
} finally {
miniCluster.startTaskExecutor();
}
}
private static int[] getMapperAttempts() {
int[] attempts = new int[MAP_NUMBER];
//noinspection StaticVariableUsedBeforeInitialization
client
.getInternalTaskInfos()
.stream()
.filter(t -> t.name.startsWith(MAP_PARTITION_TEST_PARTITION_MAPPER))
.forEach(t -> attempts[parseMapperNumberFromTaskName(t.name)] = t.attempt);
return attempts;
}
private static int parseMapperNumberFromTaskName(String name) {
Matcher m = MAPPER_NUMBER_IN_TASK_NAME_PATTERN.matcher(name);
if (m.matches()) {
return Integer.parseInt(m.group(1));
} else {
throw new FlinkRuntimeException("Failed to find mapper number in its task name: " + name);
}
}
@FunctionalInterface
private interface FailureStrategy extends Serializable {
/**
* Decides whether to fail and fails the task implicitly or by throwing an exception.
*
* @param trackingIndex index of the mapper task in the sequence
* @return {@code true} if task is failed implicitly or {@code false} if task is not failed
* @throws Exception To fail the task explicitly
*/
boolean failOrNot(int trackingIndex) throws Exception;
}
private static class OneTimeFailureStrategy implements FailureStrategy {
private static final long serialVersionUID = 1L;
private final FailureStrategy wrappedFailureStrategy;
private transient boolean failed;
private OneTimeFailureStrategy(FailureStrategy wrappedFailureStrategy) {
this.wrappedFailureStrategy = wrappedFailureStrategy;
}
@Override
public boolean failOrNot(int trackingIndex) throws Exception {
if (!failed) {
try {
boolean failedNow = wrappedFailureStrategy.failOrNot(trackingIndex);
failed = failedNow;
return failedNow;
} catch (Exception e) {
failed = true;
throw e;
}
}
return false;
}
@Override
public String toString() {
return "FailingOnce{" + wrappedFailureStrategy + '}';
}
}
private static class JoinedFailureStrategy implements FailureStrategy {
private static final long serialVersionUID = 1L;
private final FailureStrategy[] failureStrategies;
private JoinedFailureStrategy(FailureStrategy ... failureStrategies) {
this.failureStrategies = failureStrategies;
}
@Override
public boolean failOrNot(int trackingIndex) throws Exception {
for (FailureStrategy failureStrategy : failureStrategies) {
if (failureStrategy.failOrNot(trackingIndex)) {
return true;
}
}
return false;
}
@Override
public String toString() {
return String.join(
" or ",
(Iterable<String>) () -> Arrays.stream(failureStrategies).map(Object::toString).iterator());
}
}
private static class GloballyTrackingFailureStrategy implements FailureStrategy {
private static final long serialVersionUID = 1L;
private final FailureStrategy wrappedFailureStrategy;
private GloballyTrackingFailureStrategy(FailureStrategy wrappedFailureStrategy) {
this.wrappedFailureStrategy = wrappedFailureStrategy;
}
@Override
public boolean failOrNot(int trackingIndex) throws Exception {
return failureTracker.failOrNot(
trackingIndex,
wrappedFailureStrategy);
}
@Override
public String toString() {
return "Tracked{" + wrappedFailureStrategy + '}';
}
}
private static class ExceptionFailureStrategy extends AbstractOnceAfterCallNumberFailureStrategy {
private static final long serialVersionUID = 1L;
private ExceptionFailureStrategy(int failAfterCallNumber) {
super(failAfterCallNumber);
}
@Override
void fail(int trackingIndex) throws FlinkException {
throw new FlinkException("BAGA-BOOM!!! The user function generated test failure.");
}
}
private static class TaskExecutorFailureStrategy extends AbstractOnceAfterCallNumberFailureStrategy {
private static final long serialVersionUID = 1L;
private TaskExecutorFailureStrategy(int failAfterCallNumber) {
super(failAfterCallNumber);
}
@Override
void fail(int trackingIndex) throws Exception {
//noinspection OverlyBroadCatchBlock
try {
restartTaskManager();
} catch (InterruptedException e) {
// ignore the exception, task should have been failed while stopping TM
Thread.currentThread().interrupt();
} catch (Throwable t) {
failureTracker.unrelatedFailure(t);
throw t;
}
}
}
private abstract static class AbstractOnceAfterCallNumberFailureStrategy implements FailureStrategy {
private static final long serialVersionUID = 1L;
private final UUID id;
private final int failAfterCallNumber;
private transient int callCounter;
private AbstractOnceAfterCallNumberFailureStrategy(int failAfterCallNumber) {
this.failAfterCallNumber = failAfterCallNumber;
id = UUID.randomUUID();
}
@Override
public boolean failOrNot(int trackingIndex) throws Exception {
callCounter++;
boolean generateFailure = callCounter == failAfterCallNumber;
if (generateFailure) {
fail(trackingIndex);
}
return generateFailure;
}
abstract void fail(int trackingIndex) throws Exception;
@Override
public String toString() {
return this.getClass().getSimpleName() + " (fail after " + failAfterCallNumber + " calls)";
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
return Objects.equals(id, ((AbstractOnceAfterCallNumberFailureStrategy) o).id);
}
@Override
public int hashCode() {
return id.hashCode();
}
}
private static class GlobalMapFailureTracker {
private final List<Set<FailureStrategy>> mapFailures;
private final Object classLock = new Object();
@GuardedBy("classLock")
private Throwable unexpectedFailure;
private GlobalMapFailureTracker(int numberOfMappers) {
mapFailures = new ArrayList<>(numberOfMappers);
IntStream.range(0, numberOfMappers).forEach(i -> addNewMapper());
}
private int addNewMapper() {
mapFailures.add(new HashSet<>(2));
return mapFailures.size() - 1;
}
private boolean failOrNot(int index, FailureStrategy failureStrategy) throws Exception {
boolean alreadyFailed = mapFailures.get(index).contains(failureStrategy);
boolean failedNow = false;
try {
failedNow = !alreadyFailed && failureStrategy.failOrNot(index);
} catch (Exception e) {
failedNow = true;
throw e;
} finally {
if (failedNow) {
mapFailures.get(index).add(failureStrategy);
}
}
return failedNow;
}
private void unrelatedFailure(Throwable failure) {
synchronized (classLock) {
unexpectedFailure = ExceptionUtils.firstOrSuppressed(failure, unexpectedFailure);
}
}
private void verify(int[] mapAttemptNumbers) {
synchronized (classLock) {
if (unexpectedFailure != null) {
throw new AssertionError("Test failed due to unexpected exception.", unexpectedFailure);
}
}
assertThat(mapAttemptNumbers, is(EXPECTED_MAP_ATTEMPT_NUMBERS));
}
}
private static class TestPartitionMapper extends RichMapPartitionFunction<Long, Long> {
private static final long serialVersionUID = 1L;
private final int trackingIndex;
private final FailureStrategy failureStrategy;
private TestPartitionMapper(int trackingIndex, FailureStrategy failureStrategy) {
this.trackingIndex = trackingIndex;
this.failureStrategy = failureStrategy;
}
@Override
public void mapPartition(Iterable<Long> values, Collector<Long> out) throws Exception {
for (Long value : values) {
failureStrategy.failOrNot(trackingIndex);
out.collect(value + 1);
}
}
}
private static class MiniClusterClient implements AutoCloseable {
private final RestClient restClient;
private final ExecutorService executorService;
private final URI restAddress;
private MiniClusterClient(TestingMiniCluster miniCluster) throws ConfigurationException {
restAddress = miniCluster.getRestAddress().join();
executorService = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("Flink-RestClient-IO"));
restClient = createRestClient();
}
private RestClient createRestClient() throws ConfigurationException {
return new RestClient(
RestClientConfiguration.fromConfiguration(new UnmodifiableConfiguration(new Configuration())),
executorService);
}
private List<InternalTaskInfo> getInternalTaskInfos() {
return getJobs()
.stream()
.flatMap(jobId -> getJobDetails(jobId).join()
.getJobVertexInfos()
.stream()
.map(info -> Tuple2.of(jobId, info)))
.flatMap(vertexInfoWithJobId ->
getJobVertexDetailsInfo(vertexInfoWithJobId.f0, vertexInfoWithJobId.f1.getJobVertexID())
.getSubtasks()
.stream()
.map(subtask -> new InternalTaskInfo(vertexInfoWithJobId.f1.getName(), subtask)))
.collect(Collectors.toList());
}
private Collection<JobID> getJobs() {
JobIdsWithStatusOverview jobIds =
sendRequest(
JobIdsWithStatusesOverviewHeaders.getInstance(),
EmptyMessageParameters.getInstance())
.join();
return jobIds
.getJobsWithStatus()
.stream()
.map(JobIdWithStatus::getJobId)
.collect(Collectors.toList());
}
private CompletableFuture<JobDetailsInfo> getJobDetails(JobID jobId) {
JobMessageParameters params = new JobMessageParameters();
params.jobPathParameter.resolve(jobId);
return sendRequest(JobDetailsHeaders.getInstance(), params);
}
private JobVertexDetailsInfo getJobVertexDetailsInfo(JobID jobId, JobVertexID jobVertexID) {
JobVertexDetailsHeaders detailsHeaders = JobVertexDetailsHeaders.getInstance();
JobVertexMessageParameters params = new JobVertexMessageParameters();
params.jobPathParameter.resolve(jobId);
params.jobVertexIdPathParameter.resolve(jobVertexID);
return sendRequest(detailsHeaders, params).join();
}
private <M extends MessageHeaders<EmptyRequestBody, P, U>, U extends MessageParameters, P extends ResponseBody> CompletableFuture<P>
sendRequest(M messageHeaders, U messageParameters) {
try {
return restClient.sendRequest(
restAddress.getHost(),
restAddress.getPort(),
messageHeaders,
messageParameters,
EmptyRequestBody.getInstance());
} catch (IOException e) {
return FutureUtils.completedExceptionally(e);
}
}
@Override
public void close() throws Exception {
restClient.close();
executorService.shutdownNow();
}
}
private static class InternalTaskInfo {
private final String name;
private final int attempt;
private InternalTaskInfo(String name, SubtaskExecutionAttemptDetailsInfo vertexTaskDetail) {
this.name = name;
this.attempt = vertexTaskDetail.getAttempt();
}
@Override
public String toString() {
return name + " (Attempt #" + attempt + ')';
}
}
}