Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-19135] Strip ExecutionException in (Stream)ExecutionEnvironmen…
…t.execute() In FLINK-14850 we changed the execute() method to be basically final JobClient jobClient = executeAsync(...); return jobClient.getJobExecutionResult(userClassloader).get(); Unfortunately, this means that execute() now throws an ExecutionException instead of a ProgramInvocationException or JobExecutionException as before. The ExecutionException is wrapping the other exceptions that we were throwing before. We didn't notice this in tests because most tests use Test(Stream)Environment which overrides the execute() method and so doesn't go through the PipelineExecutor logic or the normal code path of delegating to executeAsync(). This change brings us back to the previous behaviour.
- Loading branch information
Showing
4 changed files
with
90 additions
and
5 deletions.
There are no files selected for viewing
This file contains 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
This file contains 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
This file contains 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
75 changes: 75 additions & 0 deletions
75
...s/src/test/java/org/apache/flink/test/streaming/api/StreamExecutionEnvironmentITCase.java
This file contains 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 |
---|---|---|
@@ -0,0 +1,75 @@ | ||
/* | ||
* 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.streaming.api; | ||
|
||
import org.apache.flink.client.deployment.executors.RemoteExecutor; | ||
import org.apache.flink.client.program.ProgramInvocationException; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.DeploymentOptions; | ||
import org.apache.flink.configuration.UnmodifiableConfiguration; | ||
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; | ||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
import org.apache.flink.test.util.MiniClusterWithClientResource; | ||
|
||
import org.junit.ClassRule; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.rules.ExpectedException; | ||
|
||
/** | ||
* Integration tests for {@link StreamExecutionEnvironment}. | ||
*/ | ||
public class StreamExecutionEnvironmentITCase { | ||
|
||
// We use our own miniClusterResource because we wan't to connect to it using a remote executor. | ||
@ClassRule | ||
public static MiniClusterWithClientResource miniClusterResource = new MiniClusterWithClientResource( | ||
new MiniClusterResourceConfiguration.Builder() | ||
.setNumberTaskManagers(1) | ||
.setNumberSlotsPerTaskManager(1) | ||
.build()); | ||
|
||
@Rule | ||
public ExpectedException thrown = ExpectedException.none(); | ||
|
||
@Test | ||
public void executeThrowsProgramInvocationException() throws Exception { | ||
UnmodifiableConfiguration clientConfiguration = miniClusterResource.getClientConfiguration(); | ||
Configuration config = new Configuration(clientConfiguration); | ||
config.set(DeploymentOptions.TARGET, RemoteExecutor.NAME); | ||
config.setBoolean(DeploymentOptions.ATTACHED, true); | ||
|
||
// Create the execution environment explicitly from a Configuration so we know that we | ||
// don't get some other subclass. If we just did | ||
// StreamExecutionEnvironment.getExecutionEnvironment() we would get a | ||
// TestStreamEnvironment that the MiniClusterResource created. We want to test the behaviour | ||
// of the base environment, though. | ||
StreamExecutionEnvironment env = | ||
new StreamExecutionEnvironment(config); | ||
|
||
env | ||
.fromElements("hello") | ||
.map(in -> { | ||
throw new RuntimeException("Failing"); | ||
}) | ||
.print(); | ||
|
||
thrown.expect(ProgramInvocationException.class); | ||
env.execute(); | ||
} | ||
} |