Skip to content

Commit

Permalink
[FLINK-25096] Fixes empty exception history for JobInitializationExce…
Browse files Browse the repository at this point in the history
…ption (#18036)
  • Loading branch information
XComp authored Dec 10, 2021
1 parent 54aaffc commit 8c3b96e
Show file tree
Hide file tree
Showing 4 changed files with 176 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,13 @@ public class ExecutionGraphInfo implements Serializable {
private final Iterable<RootExceptionHistoryEntry> exceptionHistory;

public ExecutionGraphInfo(ArchivedExecutionGraph executionGraph) {
this(executionGraph, Collections.emptyList());
this(
executionGraph,
executionGraph.getFailureInfo() != null
? Collections.singleton(
RootExceptionHistoryEntry.fromGlobalFailure(
executionGraph.getFailureInfo()))
: Collections.emptyList());
}

public ExecutionGraphInfo(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,14 @@
package org.apache.flink.runtime.scheduler.exceptionhistory;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.runtime.executiongraph.ErrorInfo;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.util.Preconditions;

import javax.annotation.Nullable;

import java.util.Collections;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
Expand Down Expand Up @@ -84,6 +87,24 @@ public static RootExceptionHistoryEntry fromGlobalFailure(
return createRootExceptionHistoryEntry(cause, timestamp, null, null, executions);
}

/**
* Creates a {@code RootExceptionHistoryEntry} based on the passed {@link ErrorInfo}. No
* concurrent failures will be added.
*
* @param errorInfo The failure information that shall be used to initialize the {@code
* RootExceptionHistoryEntry}.
* @return The {@code RootExceptionHistoryEntry} instance.
* @throws NullPointerException if {@code errorInfo} is {@code null} or the passed info does not
* contain a {@code Throwable}.
* @throws IllegalArgumentException if the passed {@code timestamp} is not bigger than {@code
* 0}.
*/
public static RootExceptionHistoryEntry fromGlobalFailure(ErrorInfo errorInfo) {
Preconditions.checkNotNull(errorInfo, "errorInfo");
return fromGlobalFailure(
errorInfo.getException(), errorInfo.getTimestamp(), Collections.emptyList());
}

private static RootExceptionHistoryEntry createRootExceptionHistoryEntry(
Throwable cause,
long timestamp,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,26 +20,37 @@

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.core.testutils.FlinkMatchers;
import org.apache.flink.runtime.client.JobInitializationException;
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.executiongraph.ErrorInfo;
import org.apache.flink.runtime.jobmaster.factories.TestingJobMasterServiceFactory;
import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder;
import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
import org.apache.flink.runtime.scheduler.exceptionhistory.RootExceptionHistoryEntry;
import org.apache.flink.util.SerializedThrowable;
import org.apache.flink.util.TestLogger;

import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;

import org.hamcrest.CoreMatchers;
import org.junit.Test;

import java.time.Duration;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.function.Function;

import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage;
import static org.apache.flink.core.testutils.FlinkMatchers.futureWillCompleteExceptionally;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.notNullValue;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
Expand Down Expand Up @@ -70,6 +81,77 @@ public void testInitializationFailureCompletesResultFuture() {
assertThat(initializationFailure, containsCause(originalCause));
}

@Test
public void testInitializationFailureSetsFailureInfoProperly()
throws ExecutionException, InterruptedException {
final CompletableFuture<JobMasterService> jobMasterServiceFuture =
new CompletableFuture<>();
DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture);
final RuntimeException originalCause = new RuntimeException("Expected RuntimeException");

long beforeFailureTimestamp = System.currentTimeMillis();
jobMasterServiceFuture.completeExceptionally(originalCause);
long afterFailureTimestamp = System.currentTimeMillis();

final JobManagerRunnerResult result = serviceProcess.getResultFuture().get();
final ErrorInfo executionGraphFailure =
result.getExecutionGraphInfo().getArchivedExecutionGraph().getFailureInfo();

assertThat(executionGraphFailure, is(notNullValue()));
assertInitializationException(
executionGraphFailure.getException(),
originalCause,
executionGraphFailure.getTimestamp(),
beforeFailureTimestamp,
afterFailureTimestamp);
}

@Test
public void testInitializationFailureSetsExceptionHistoryProperly()
throws ExecutionException, InterruptedException {
final CompletableFuture<JobMasterService> jobMasterServiceFuture =
new CompletableFuture<>();
DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture);
final RuntimeException originalCause = new RuntimeException("Expected RuntimeException");

long beforeFailureTimestamp = System.currentTimeMillis();
jobMasterServiceFuture.completeExceptionally(originalCause);
long afterFailureTimestamp = System.currentTimeMillis();

final RootExceptionHistoryEntry entry =
Iterables.getOnlyElement(
serviceProcess
.getResultFuture()
.get()
.getExecutionGraphInfo()
.getExceptionHistory());

assertInitializationException(
entry.getException(),
originalCause,
entry.getTimestamp(),
beforeFailureTimestamp,
afterFailureTimestamp);
assertThat(entry.isGlobal(), is(true));
}

private static void assertInitializationException(
SerializedThrowable actualException,
Throwable expectedCause,
long actualTimestamp,
long expectedLowerTimestampThreshold,
long expectedUpperTimestampThreshold) {
final Throwable deserializedException =
actualException.deserializeError(Thread.currentThread().getContextClassLoader());

assertThat(
deserializedException, CoreMatchers.instanceOf(JobInitializationException.class));
assertThat(deserializedException, FlinkMatchers.containsCause(expectedCause));

assertThat(actualTimestamp, greaterThanOrEqualTo(expectedLowerTimestampThreshold));
assertThat(actualTimestamp, lessThanOrEqualTo(expectedUpperTimestampThreshold));
}

@Test
public void testCloseAfterInitializationFailure() throws Exception {
final CompletableFuture<JobMasterService> jobMasterServiceFuture =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
/*
* 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.runtime.scheduler;

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.executiongraph.ErrorInfo;
import org.apache.flink.runtime.scheduler.exceptionhistory.RootExceptionHistoryEntry;

import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;

import org.junit.Test;

import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.notNullValue;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat;

/** {@code ExecutionGraphInfoTest} tests the proper initialization of {@link ExecutionGraphInfo}. */
public class ExecutionGraphInfoTest {

@Test
public void testExecutionGraphHistoryBeingDerivedFromFailedExecutionGraph() {
final ArchivedExecutionGraph executionGraph =
ArchivedExecutionGraph.createFromInitializingJob(
new JobID(),
"test job name",
JobStatus.FAILED,
new RuntimeException("Expected RuntimeException"),
null,
System.currentTimeMillis());

final ExecutionGraphInfo executionGraphInfo = new ExecutionGraphInfo(executionGraph);

final ErrorInfo failureInfo =
executionGraphInfo.getArchivedExecutionGraph().getFailureInfo();

final RootExceptionHistoryEntry actualEntry =
Iterables.getOnlyElement(executionGraphInfo.getExceptionHistory());

assertThat(failureInfo, is(notNullValue()));
assertThat(failureInfo.getException(), is(actualEntry.getException()));
assertThat(failureInfo.getTimestamp(), is(actualEntry.getTimestamp()));

assertThat(actualEntry.isGlobal(), is(true));
assertThat(actualEntry.getFailingTaskName(), is(nullValue()));
assertThat(actualEntry.getTaskManagerLocation(), is(nullValue()));
}
}

0 comments on commit 8c3b96e

Please sign in to comment.