Skip to content

Commit

Permalink
[FLINK-19135] Strip ExecutionException in (Stream)ExecutionEnvironmen…
Browse files Browse the repository at this point in the history
…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
aljoscha committed Sep 8, 2020
1 parent 22812c1 commit 0b67901
Show file tree
Hide file tree
Showing 4 changed files with 90 additions and 5 deletions.
Expand Up @@ -899,10 +899,15 @@ public JobExecutionResult execute(String jobName) throws Exception {
jobListener -> jobListener.onJobExecuted(lastJobExecutionResult, null));

} catch (Throwable t) {
// get() on the JobExecutionResult Future will throw an ExecutionException. This
// behaviour was largely not there in Flink versions before the PipelineExecutor
// refactoring so we should strip that exception.
Throwable strippedException = ExceptionUtils.stripExecutionException(t);

jobListeners.forEach(jobListener -> {
jobListener.onJobExecuted(null, ExceptionUtils.stripExecutionException(t));
jobListener.onJobExecuted(null, strippedException);
});
ExceptionUtils.rethrowException(t);
ExceptionUtils.rethrowException(strippedException);
}

return lastJobExecutionResult;
Expand Down
Expand Up @@ -1712,10 +1712,15 @@ public JobExecutionResult execute(StreamGraph streamGraph) throws Exception {

return jobExecutionResult;
} catch (Throwable t) {
// get() on the JobExecutionResult Future will throw an ExecutionException. This
// behaviour was largely not there in Flink versions before the PipelineExecutor
// refactoring so we should strip that exception.
Throwable strippedException = ExceptionUtils.stripExecutionException(t);

jobListeners.forEach(jobListener -> {
jobListener.onJobExecuted(null, ExceptionUtils.stripExecutionException(t));
jobListener.onJobExecuted(null, strippedException);
});
ExceptionUtils.rethrowException(t);
ExceptionUtils.rethrowException(strippedException);

// never reached, only make javac happy
return null;
Expand Down
Expand Up @@ -233,7 +233,7 @@ public Long map(Long value) throws Exception {
Throwable error = errorRef[0];
assertNotNull("The program did not fail properly", error);

assertTrue(error.getCause() instanceof ProgramInvocationException);
assertTrue(error instanceof ProgramInvocationException);
// all seems well :-)
}
catch (Exception e) {
Expand Down
@@ -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();
}
}

0 comments on commit 0b67901

Please sign in to comment.