-
Notifications
You must be signed in to change notification settings - Fork 13k
/
YarnTestBase.java
820 lines (709 loc) · 28.3 KB
/
YarnTestBase.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
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
/*
* 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.yarn;
import org.apache.flink.client.cli.CliFrontend;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.SecurityOptions;
import org.apache.flink.test.util.TestBaseUtils;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.TestLogger;
import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.service.Service;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.MiniYARNCluster;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.Marker;
import org.slf4j.MarkerFactory;
import java.io.BufferedWriter;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileWriter;
import java.io.FilenameFilter;
import java.io.IOException;
import java.io.InputStream;
import java.io.PipedInputStream;
import java.io.PipedOutputStream;
import java.io.PrintStream;
import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Scanner;
import java.util.concurrent.ConcurrentMap;
import java.util.regex.Pattern;
/**
* This base class allows to use the MiniYARNCluster.
* The cluster is re-used for all tests.
*
* <p>This class is located in a different package which is build after flink-dist. This way,
* we can use the YARN uberjar of flink to start a Flink YARN session.
*
* <p>The test is not thread-safe. Parallel execution of tests is not possible!
*/
public abstract class YarnTestBase extends TestLogger {
private static final Logger LOG = LoggerFactory.getLogger(YarnTestBase.class);
protected static final PrintStream ORIGINAL_STDOUT = System.out;
protected static final PrintStream ORIGINAL_STDERR = System.err;
private static final InputStream ORIGINAL_STDIN = System.in;
protected static final String TEST_CLUSTER_NAME_KEY = "flink-yarn-minicluster-name";
protected static final int NUM_NODEMANAGERS = 2;
/** The tests are scanning for these strings in the final output. */
protected static final String[] PROHIBITED_STRINGS = {
"Exception", // we don't want any exceptions to happen
"Started SelectChannelConnector@0.0.0.0:8081" // Jetty should start on a random port in YARN mode.
};
/** These strings are white-listed, overriding the prohibited strings. */
protected static final String[] WHITELISTED_STRINGS = {
"akka.remote.RemoteTransportExceptionNoStackTrace",
// workaround for annoying InterruptedException logging:
// https://issues.apache.org/jira/browse/YARN-1022
"java.lang.InterruptedException",
// very specific on purpose
"Remote connection to [null] failed with java.net.ConnectException: Connection refused",
"Remote connection to [null] failed with java.nio.channels.NotYetConnectedException",
"java.io.IOException: Connection reset by peer",
// this can happen in Akka 2.4 on shutdown.
"java.util.concurrent.RejectedExecutionException: Worker has already been shutdown"
};
// Temp directory which is deleted after the unit test.
@ClassRule
public static TemporaryFolder tmp = new TemporaryFolder();
protected static MiniYARNCluster yarnCluster = null;
/**
* Uberjar (fat jar) file of Flink.
*/
protected static File flinkUberjar;
protected static final YarnConfiguration YARN_CONFIGURATION;
/**
* lib/ folder of the flink distribution.
*/
protected static File flinkLibFolder;
/**
* Temporary folder where Flink configurations will be kept for secure run.
*/
protected static File tempConfPathForSecureRun = null;
static {
YARN_CONFIGURATION = new YarnConfiguration();
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096); // 4096 is the available memory anyways
YARN_CONFIGURATION.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true);
YARN_CONFIGURATION.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true);
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2);
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 4);
YARN_CONFIGURATION.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600);
YARN_CONFIGURATION.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false);
YARN_CONFIGURATION.setInt(YarnConfiguration.NM_VCORES, 666); // memory is overwritten in the MiniYARNCluster.
// so we have to change the number of cores for testing.
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 20000); // 20 seconds expiry (to ensure we properly heartbeat with YARN).
}
public static void populateYarnSecureConfigurations(Configuration conf, String principal, String keytab) {
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, "true");
conf.set(YarnConfiguration.RM_KEYTAB, keytab);
conf.set(YarnConfiguration.RM_PRINCIPAL, principal);
conf.set(YarnConfiguration.NM_KEYTAB, keytab);
conf.set(YarnConfiguration.NM_PRINCIPAL, principal);
conf.set(YarnConfiguration.RM_WEBAPP_SPNEGO_USER_NAME_KEY, principal);
conf.set(YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY, keytab);
conf.set(YarnConfiguration.NM_WEBAPP_SPNEGO_USER_NAME_KEY, principal);
conf.set(YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY, keytab);
conf.set("hadoop.security.auth_to_local", "RULE:[1:$1] RULE:[2:$1]");
}
/**
* Sleep a bit between the tests (we are re-using the YARN cluster for the tests).
*/
@After
public void sleep() {
try {
Thread.sleep(500);
} catch (InterruptedException e) {
Assert.fail("Should not happen");
}
}
private YarnClient yarnClient = null;
protected org.apache.flink.configuration.Configuration flinkConfiguration;
protected boolean flip6;
@Before
public void checkClusterEmpty() throws IOException, YarnException {
if (yarnClient == null) {
yarnClient = YarnClient.createYarnClient();
yarnClient.init(YARN_CONFIGURATION);
yarnClient.start();
}
List<ApplicationReport> apps = yarnClient.getApplications();
for (ApplicationReport app : apps) {
if (app.getYarnApplicationState() != YarnApplicationState.FINISHED
&& app.getYarnApplicationState() != YarnApplicationState.KILLED
&& app.getYarnApplicationState() != YarnApplicationState.FAILED) {
Assert.fail("There is at least one application on the cluster is not finished." +
"App " + app.getApplicationId() + " is in state " + app.getYarnApplicationState());
}
}
flinkConfiguration = new org.apache.flink.configuration.Configuration();
flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE));
}
protected YarnClient getYarnClient() {
return yarnClient;
}
protected static YarnConfiguration getYarnConfiguration() {
return YARN_CONFIGURATION;
}
/**
* Locate a file or directory.
*/
public static File findFile(String startAt, FilenameFilter fnf) {
File root = new File(startAt);
String[] files = root.list();
if (files == null) {
return null;
}
for (String file : files) {
File f = new File(startAt + File.separator + file);
if (f.isDirectory()) {
File r = findFile(f.getAbsolutePath(), fnf);
if (r != null) {
return r;
}
} else if (fnf.accept(f.getParentFile(), f.getName())) {
return f;
}
}
return null;
}
/**
* Filter to find root dir of the flink-yarn dist.
*/
public static class RootDirFilenameFilter implements FilenameFilter {
@Override
public boolean accept(File dir, String name) {
return name.startsWith("flink-dist") && name.endsWith(".jar") && dir.toString().contains("/lib");
}
}
/**
* A simple {@link FilenameFilter} that only accepts files if their name contains every string in the array passed
* to the constructor.
*/
public static class ContainsName implements FilenameFilter {
private String[] names;
private String excludeInPath = null;
/**
* @param names which have to be included in the filename.
*/
public ContainsName(String[] names) {
this.names = names;
}
public ContainsName(String[] names, String excludeInPath) {
this.names = names;
this.excludeInPath = excludeInPath;
}
@Override
public boolean accept(File dir, String name) {
if (excludeInPath == null) {
for (String n: names) {
if (!name.contains(n)) {
return false;
}
}
return true;
} else {
for (String n: names) {
if (!name.contains(n)) {
return false;
}
}
return !dir.toString().contains(excludeInPath);
}
}
}
public static File writeYarnSiteConfigXML(Configuration yarnConf) throws IOException {
tmp.create();
File yarnSiteXML = new File(tmp.newFolder().getAbsolutePath() + "/yarn-site.xml");
try (FileWriter writer = new FileWriter(yarnSiteXML)) {
yarnConf.writeXml(writer);
writer.flush();
}
return yarnSiteXML;
}
/**
* This method checks the written TaskManager and JobManager log files
* for exceptions.
*
* <p>WARN: Please make sure the tool doesn't find old logfiles from previous test runs.
* So always run "mvn clean" before running the tests here.
*
*/
public static void ensureNoProhibitedStringInLogFiles(final String[] prohibited, final String[] whitelisted) {
File cwd = new File("target/" + YARN_CONFIGURATION.get(TEST_CLUSTER_NAME_KEY));
Assert.assertTrue("Expecting directory " + cwd.getAbsolutePath() + " to exist", cwd.exists());
Assert.assertTrue("Expecting directory " + cwd.getAbsolutePath() + " to be a directory", cwd.isDirectory());
List<String> prohibitedExcerpts = new ArrayList<>();
File foundFile = findFile(cwd.getAbsolutePath(), new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
// scan each file for prohibited strings.
File f = new File(dir.getAbsolutePath() + "/" + name);
try {
Scanner scanner = new Scanner(f);
while (scanner.hasNextLine()) {
final String lineFromFile = scanner.nextLine();
for (String aProhibited : prohibited) {
if (lineFromFile.contains(aProhibited)) {
boolean whitelistedFound = false;
for (String white : whitelisted) {
if (lineFromFile.contains(white)) {
whitelistedFound = true;
break;
}
}
if (!whitelistedFound) {
// logging in FATAL to see the actual message in TRAVIS tests.
Marker fatal = MarkerFactory.getMarker("FATAL");
LOG.error(fatal, "Prohibited String '{}' in line '{}'", aProhibited, lineFromFile);
StringBuilder logExcerpt = new StringBuilder();
logExcerpt.append(System.lineSeparator());
logExcerpt.append(lineFromFile);
logExcerpt.append(System.lineSeparator());
// extract potential stack trace from log
while (scanner.hasNextLine()) {
String line = scanner.nextLine();
if (!line.isEmpty() && (Character.isWhitespace(line.charAt(0)) || line.startsWith("Caused by"))) {
logExcerpt.append(line);
logExcerpt.append(System.lineSeparator());
} else {
break;
}
}
prohibitedExcerpts.add(logExcerpt.toString());
return true;
}
}
}
}
} catch (FileNotFoundException e) {
LOG.warn("Unable to locate file: " + e.getMessage() + " file: " + f.getAbsolutePath());
}
return false;
}
});
if (foundFile != null) {
Scanner scanner = null;
try {
scanner = new Scanner(foundFile);
} catch (FileNotFoundException e) {
Assert.fail("Unable to locate file: " + e.getMessage() + " file: " + foundFile.getAbsolutePath());
}
LOG.warn("Found a file with a prohibited string. Printing contents:");
while (scanner.hasNextLine()) {
LOG.warn("LINE: " + scanner.nextLine());
}
Assert.fail(
"Found a file " + foundFile + " with a prohibited string (one of " + Arrays.toString(prohibited) + "). " +
"Excerpts:" + System.lineSeparator() + prohibitedExcerpts);
}
}
public static void sleep(int time) {
try {
Thread.sleep(time);
} catch (InterruptedException e) {
LOG.warn("Interruped", e);
}
}
public static int getRunningContainers() {
int count = 0;
for (int nmId = 0; nmId < NUM_NODEMANAGERS; nmId++) {
NodeManager nm = yarnCluster.getNodeManager(nmId);
ConcurrentMap<ContainerId, Container> containers = nm.getNMContext().getContainers();
count += containers.size();
}
return count;
}
public static void startYARNSecureMode(Configuration conf, String principal, String keytab) {
start(conf, principal, keytab);
}
public static void startYARNWithConfig(Configuration conf) {
start(conf, null, null);
}
private static void start(Configuration conf, String principal, String keytab) {
// set the home directory to a temp directory. Flink on YARN is using the home dir to distribute the file
File homeDir = null;
try {
homeDir = tmp.newFolder();
} catch (IOException e) {
e.printStackTrace();
Assert.fail(e.getMessage());
}
System.setProperty("user.home", homeDir.getAbsolutePath());
String uberjarStartLoc = "..";
LOG.info("Trying to locate uberjar in {}", new File(uberjarStartLoc));
flinkUberjar = findFile(uberjarStartLoc, new RootDirFilenameFilter());
Assert.assertNotNull("Flink uberjar not found", flinkUberjar);
String flinkDistRootDir = flinkUberjar.getParentFile().getParent();
flinkLibFolder = flinkUberjar.getParentFile(); // the uberjar is located in lib/
Assert.assertNotNull("Flink flinkLibFolder not found", flinkLibFolder);
Assert.assertTrue("lib folder not found", flinkLibFolder.exists());
Assert.assertTrue("lib folder not found", flinkLibFolder.isDirectory());
if (!flinkUberjar.exists()) {
Assert.fail("Unable to locate yarn-uberjar.jar");
}
try {
LOG.info("Starting up MiniYARNCluster");
if (yarnCluster == null) {
yarnCluster = new MiniYARNCluster(conf.get(YarnTestBase.TEST_CLUSTER_NAME_KEY), NUM_NODEMANAGERS, 1, 1);
yarnCluster.init(conf);
yarnCluster.start();
}
Map<String, String> map = new HashMap<String, String>(System.getenv());
File flinkConfDirPath = findFile(flinkDistRootDir, new ContainsName(new String[]{"flink-conf.yaml"}));
Assert.assertNotNull(flinkConfDirPath);
if (!StringUtils.isBlank(principal) && !StringUtils.isBlank(keytab)) {
//copy conf dir to test temporary workspace location
tempConfPathForSecureRun = tmp.newFolder("conf");
String confDirPath = flinkConfDirPath.getParentFile().getAbsolutePath();
FileUtils.copyDirectory(new File(confDirPath), tempConfPathForSecureRun);
try (FileWriter fw = new FileWriter(new File(tempConfPathForSecureRun, "flink-conf.yaml"), true);
BufferedWriter bw = new BufferedWriter(fw);
PrintWriter out = new PrintWriter(bw)) {
LOG.info("writing keytab: " + keytab + " and principal: " + principal + " to config file");
out.println("");
out.println("#Security Configurations Auto Populated ");
out.println(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key() + ": " + keytab);
out.println(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key() + ": " + principal);
out.println("");
} catch (IOException e) {
throw new RuntimeException("Exception occured while trying to append the security configurations.", e);
}
String configDir = tempConfPathForSecureRun.getAbsolutePath();
LOG.info("Temporary Flink configuration directory to be used for secure test: {}", configDir);
Assert.assertNotNull(configDir);
map.put(ConfigConstants.ENV_FLINK_CONF_DIR, configDir);
} else {
map.put(ConfigConstants.ENV_FLINK_CONF_DIR, flinkConfDirPath.getParent());
}
File yarnConfFile = writeYarnSiteConfigXML(conf);
map.put("YARN_CONF_DIR", yarnConfFile.getParentFile().getAbsolutePath());
map.put("IN_TESTS", "yes we are in tests"); // see YarnClusterDescriptor() for more infos
TestBaseUtils.setEnv(map);
Assert.assertTrue(yarnCluster.getServiceState() == Service.STATE.STARTED);
// wait for the nodeManagers to connect
while (!yarnCluster.waitForNodeManagersToConnect(500)) {
LOG.info("Waiting for Nodemanagers to connect");
}
} catch (Exception ex) {
ex.printStackTrace();
LOG.error("setup failure", ex);
Assert.fail();
}
}
/**
* Default @BeforeClass impl. Overwrite this for passing a different configuration
*/
@BeforeClass
public static void setup() {
startYARNWithConfig(YARN_CONFIGURATION);
}
// -------------------------- Runner -------------------------- //
protected static ByteArrayOutputStream outContent;
protected static ByteArrayOutputStream errContent;
enum RunTypes {
YARN_SESSION, CLI_FRONTEND
}
/**
* This method returns once the "startedAfterString" has been seen.
*/
protected Runner startWithArgs(String[] args, String startedAfterString, RunTypes type) throws IOException {
LOG.info("Running with args {}", Arrays.toString(args));
outContent = new ByteArrayOutputStream();
errContent = new ByteArrayOutputStream();
PipedOutputStream out = new PipedOutputStream();
PipedInputStream in = new PipedInputStream(out);
PrintStream stdinPrintStream = new PrintStream(out);
System.setOut(new PrintStream(outContent));
System.setErr(new PrintStream(errContent));
System.setIn(in);
final int startTimeoutSeconds = 60;
Runner runner = new Runner(
args,
flinkConfiguration,
CliFrontend.getConfigurationDirectoryFromEnv(),
type,
0,
stdinPrintStream);
runner.setName("Frontend (CLI/YARN Client) runner thread (startWithArgs()).");
runner.start();
for (int second = 0; second < startTimeoutSeconds; second++) {
sleep(1000);
// check output for correct TaskManager startup.
if (outContent.toString().contains(startedAfterString)
|| errContent.toString().contains(startedAfterString)) {
LOG.info("Found expected output in redirected streams");
return runner;
}
// check if thread died
if (!runner.isAlive()) {
resetStreamsAndSendOutput();
if (runner.getRunnerError() != null) {
throw new RuntimeException("Runner failed with exception.", runner.getRunnerError());
}
Assert.fail("Runner thread died before the test was finished.");
}
}
resetStreamsAndSendOutput();
Assert.fail("During the timeout period of " + startTimeoutSeconds + " seconds the " +
"expected string did not show up");
return null;
}
protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnStrings, RunTypes type, int returnCode) throws IOException {
runWithArgs(args, terminateAfterString, failOnStrings, type, returnCode, false);
}
/**
* The test has been passed once the "terminateAfterString" has been seen.
* @param args Command line arguments for the runner
* @param terminateAfterString the runner is searching the stdout and stderr for this string. as soon as it appears, the test has passed
* @param failOnPatterns The runner is searching stdout and stderr for the pattern (regexp) specified here. If one appears, the test has failed
* @param type Set the type of the runner
* @param expectedReturnValue Expected return code from the runner.
* @param checkLogForTerminateString If true, the runner checks also the log4j logger for the terminate string
*/
protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnPatterns, RunTypes type, int expectedReturnValue, boolean checkLogForTerminateString) throws IOException {
LOG.info("Running with args {}", Arrays.toString(args));
outContent = new ByteArrayOutputStream();
errContent = new ByteArrayOutputStream();
PipedOutputStream out = new PipedOutputStream();
PipedInputStream in = new PipedInputStream(out);
PrintStream stdinPrintStream = new PrintStream(out);
System.setOut(new PrintStream(outContent));
System.setErr(new PrintStream(errContent));
System.setIn(in);
// we wait for at most three minutes
final int startTimeoutSeconds = 180;
final long deadline = System.currentTimeMillis() + (startTimeoutSeconds * 1000);
Runner runner = new Runner(
args,
flinkConfiguration,
CliFrontend.getConfigurationDirectoryFromEnv(),
type,
expectedReturnValue,
stdinPrintStream);
runner.start();
boolean expectedStringSeen = false;
boolean testPassedFromLog4j = false;
long shutdownTimeout = 30000L;
do {
sleep(1000);
String outContentString = outContent.toString();
String errContentString = errContent.toString();
if (failOnPatterns != null) {
for (String failOnString : failOnPatterns) {
Pattern pattern = Pattern.compile(failOnString);
if (pattern.matcher(outContentString).find() || pattern.matcher(errContentString).find()) {
LOG.warn("Failing test. Output contained illegal string '" + failOnString + "'");
resetStreamsAndSendOutput();
// stopping runner.
runner.sendStop();
// wait for the thread to stop
try {
runner.join(shutdownTimeout);
} catch (InterruptedException e) {
LOG.warn("Interrupted while stopping runner", e);
}
Assert.fail("Output contained illegal string '" + failOnString + "'");
}
}
}
// check output for the expected terminateAfterString.
if (checkLogForTerminateString) {
LoggingEvent matchedEvent = UtilsTest.getEventContainingString(terminateAfterString);
if (matchedEvent != null) {
testPassedFromLog4j = true;
LOG.info("Found expected output in logging event {}", matchedEvent);
}
}
if (outContentString.contains(terminateAfterString) || errContentString.contains(terminateAfterString) || testPassedFromLog4j) {
expectedStringSeen = true;
LOG.info("Found expected output in redirected streams");
// send "stop" command to command line interface
LOG.info("RunWithArgs: request runner to stop");
runner.sendStop();
// wait for the thread to stop
try {
runner.join(shutdownTimeout);
}
catch (InterruptedException e) {
LOG.warn("Interrupted while stopping runner", e);
}
LOG.warn("RunWithArgs runner stopped.");
}
else {
// check if thread died
if (!runner.isAlive()) {
// leave loop: the runner died, so we can not expect new strings to show up.
break;
}
}
}
while (runner.getRunnerError() == null && !expectedStringSeen && System.currentTimeMillis() < deadline);
resetStreamsAndSendOutput();
if (runner.getRunnerError() != null) {
// this lets the test fail.
throw new RuntimeException("Runner failed", runner.getRunnerError());
}
Assert.assertTrue("During the timeout period of " + startTimeoutSeconds + " seconds the " +
"expected string \"" + terminateAfterString + "\" did not show up.", expectedStringSeen);
LOG.info("Test was successful");
}
protected static void resetStreamsAndSendOutput() {
System.setOut(ORIGINAL_STDOUT);
System.setErr(ORIGINAL_STDERR);
System.setIn(ORIGINAL_STDIN);
LOG.info("Sending stdout content through logger: \n\n{}\n\n", outContent.toString());
LOG.info("Sending stderr content through logger: \n\n{}\n\n", errContent.toString());
}
/**
* Utility class to run yarn jobs.
*/
protected static class Runner extends Thread {
private final String[] args;
private final org.apache.flink.configuration.Configuration configuration;
private final String configurationDirectory;
private final int expectedReturnValue;
private final PrintStream stdinPrintStream;
private RunTypes type;
private FlinkYarnSessionCli yCli;
private Throwable runnerError;
public Runner(
String[] args,
org.apache.flink.configuration.Configuration configuration,
String configurationDirectory,
RunTypes type,
int expectedReturnValue,
PrintStream stdinPrintStream) {
this.args = args;
this.configuration = Preconditions.checkNotNull(configuration);
this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory);
this.type = type;
this.expectedReturnValue = expectedReturnValue;
this.stdinPrintStream = Preconditions.checkNotNull(stdinPrintStream);
}
@Override
public void run() {
try {
int returnValue;
switch (type) {
case YARN_SESSION:
yCli = new FlinkYarnSessionCli(
configuration,
configurationDirectory,
"",
"",
true);
returnValue = yCli.run(args);
break;
case CLI_FRONTEND:
try {
CliFrontend cli = new CliFrontend(
configuration,
CliFrontend.loadCustomCommandLines(configuration, configurationDirectory));
returnValue = cli.parseParameters(args);
} catch (Exception e) {
throw new RuntimeException("Failed to execute the following args with CliFrontend: "
+ Arrays.toString(args), e);
}
break;
default:
throw new RuntimeException("Unknown type " + type);
}
if (returnValue != this.expectedReturnValue) {
Assert.fail("The YARN session returned with unexpected value=" + returnValue + " expected=" + expectedReturnValue);
}
} catch (Throwable t) {
LOG.info("Runner stopped with exception", t);
// save error.
this.runnerError = t;
}
}
/** Stops the Yarn session. */
public void sendStop() {
stdinPrintStream.println("stop");
}
public Throwable getRunnerError() {
return runnerError;
}
}
// -------------------------- Tear down -------------------------- //
@AfterClass
public static void teardown() throws Exception {
LOG.info("Stopping MiniYarn Cluster");
yarnCluster.stop();
// Unset FLINK_CONF_DIR, as it might change the behavior of other tests
Map<String, String> map = new HashMap<>(System.getenv());
map.remove(ConfigConstants.ENV_FLINK_CONF_DIR);
map.remove("YARN_CONF_DIR");
map.remove("IN_TESTS");
TestBaseUtils.setEnv(map);
if (tempConfPathForSecureRun != null) {
FileUtil.fullyDelete(tempConfPathForSecureRun);
tempConfPathForSecureRun = null;
}
// When we are on travis, we copy the temp files of JUnit (containing the MiniYARNCluster log files)
// to <flinkRoot>/target/flink-yarn-tests-*.
// The files from there are picked up by the ./tools/travis_watchdog.sh script
// to upload them to Amazon S3.
if (isOnTravis()) {
File target = new File("../target" + YARN_CONFIGURATION.get(TEST_CLUSTER_NAME_KEY));
if (!target.mkdirs()) {
LOG.warn("Error creating dirs to {}", target);
}
File src = tmp.getRoot();
LOG.info("copying the final files from {} to {}", src.getAbsolutePath(), target.getAbsolutePath());
try {
FileUtils.copyDirectoryToDirectory(src, target);
} catch (IOException e) {
LOG.warn("Error copying the final files from {} to {}: msg: {}", src.getAbsolutePath(), target.getAbsolutePath(), e.getMessage(), e);
}
}
}
public static boolean isOnTravis() {
return System.getenv("TRAVIS") != null && System.getenv("TRAVIS").equals("true");
}
}