-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-22085][tests] Update TestUtils::tryExecute() to cancel the job after execution failure #15713
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
[FLINK-22085][tests] Update TestUtils::tryExecute() to cancel the job after execution failure #15713
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,41 +18,50 @@ | |
|
|
||
| package org.apache.flink.test.util; | ||
|
|
||
| import org.apache.flink.api.common.JobExecutionResult; | ||
| import org.apache.flink.api.common.JobID; | ||
| import org.apache.flink.client.ClientUtils; | ||
| import org.apache.flink.client.program.ClusterClient; | ||
| import org.apache.flink.client.program.ProgramInvocationException; | ||
| import org.apache.flink.runtime.client.JobExecutionException; | ||
| import org.apache.flink.core.execution.JobClient; | ||
| import org.apache.flink.runtime.client.JobInitializationException; | ||
| import org.apache.flink.runtime.jobgraph.JobGraph; | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.flink.streaming.api.graph.StreamGraph; | ||
|
|
||
| import static org.junit.Assert.fail; | ||
|
|
||
| /** Test utilities. */ | ||
| public class TestUtils { | ||
|
|
||
| public static JobExecutionResult tryExecute(StreamExecutionEnvironment see, String name) | ||
| throws Exception { | ||
| // Execute the job and wait for the job result synchronously. The method throws exception | ||
| // iff one of the following conditions happens: | ||
| // 1) The job finishes successfully without exception | ||
| // 2) The job finishes with an exception that contains SuccessException. | ||
| public static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception { | ||
| JobClient jobClient = null; | ||
| try { | ||
| return see.execute(name); | ||
| } catch (ProgramInvocationException | JobExecutionException root) { | ||
| Throwable cause = root.getCause(); | ||
|
|
||
| // search for nested SuccessExceptions | ||
| int depth = 0; | ||
| while (!(cause instanceof SuccessException)) { | ||
| if (cause == null || depth++ == 20) { | ||
| root.printStackTrace(); | ||
| fail("Test failed: " + root.getMessage()); | ||
| } else { | ||
| cause = cause.getCause(); | ||
| StreamGraph graph = see.getStreamGraph(name); | ||
| jobClient = see.executeAsync(graph); | ||
| jobClient.getJobExecutionResult().get(); | ||
| } catch (Throwable root) { | ||
| if (jobClient != null) { | ||
| try { | ||
| jobClient.cancel().get(); | ||
| } catch (Exception e) { | ||
| // Exception could be thrown if the job has already finished. | ||
| // Ignore the exception. | ||
| } | ||
| } | ||
| } | ||
|
|
||
| return null; | ||
| Throwable t = root; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We could reuse
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nice! |
||
| while (t != null && !(t instanceof SuccessException)) { | ||
| t = t.getCause(); | ||
| } | ||
|
|
||
| if (t == null) { | ||
| root.printStackTrace(); | ||
| fail("Test failed: " + root.getMessage()); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public static void submitJobAndWaitForResult( | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I believe the description is currently wrong. How about we rephrase it:
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you @dawidwys for catching this. The updated comment is much better.