zentol commented on a change in pull request #15311: URL: https://github.com/apache/flink/pull/15311#discussion_r601294080
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractorTest.java ########## @@ -0,0 +1,159 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables; Review comment: wrong import ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractorTest.java ########## @@ -0,0 +1,159 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Range; + +import org.hamcrest.collection.IsIterableContainingInOrder; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertThat; + +/** {@code ExceptionHistoryEntryExtractorTest} tests {@link ExceptionHistoryEntryExtractor}. */ +public class ExceptionHistoryEntryExtractorTest extends TestLogger { + + private ExecutionGraph executionGraph; + + @Before + public void setup() throws JobException, JobExecutionException { + final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); + jobGraph.getVertices().forEach(v -> v.setParallelism(3)); + + executionGraph = + TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + } + + @Test + public void extractLocalFailure() { + final Throwable rootException = new RuntimeException("Expected root failure"); + final ExecutionVertex rootExecutionVertex = extractExecutionVertex(0); + final Range<Long> rootFailureTimeframe = triggerFailure(rootExecutionVertex, rootException); + + final Throwable otherException = new IllegalStateException("Expected other failure"); + final ExecutionVertex otherExecutionVertex = extractExecutionVertex(1); Review comment: ```suggestion final ExecutionVertex concurrentlyFailedExecutionVertex = extractExecutionVertex(1); ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractor.java ########## @@ -0,0 +1,105 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.util.Preconditions; + +import java.util.Map; + +/** + * {@code ExceptionHistoryEntryExtractor} extracts all the necessary information from given + * executions to create corresponding {@link RootExceptionHistoryEntry RootExceptionHistoryEntries}. + */ +public class ExceptionHistoryEntryExtractor { + + public RootExceptionHistoryEntry extractLocalFailure( + Map<JobVertexID, ExecutionJobVertex> executionJobVertices, + ExecutionVertexID executionVertexId, + Iterable<ExecutionVertexID> otherAffectedVertices) { + final ExceptionHistoryEntry entry = + extractExceptionHistoryEntry(executionJobVertices, executionVertexId); + + Preconditions.checkArgument( + entry != null, + "The passed ExecutionVertexId {} does not correspond to a failed execution.", Review comment: this can be misleading; e.g., if executionJobVertices does not contain the vertex. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractor.java ########## @@ -0,0 +1,105 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.util.Preconditions; + +import java.util.Map; + +/** + * {@code ExceptionHistoryEntryExtractor} extracts all the necessary information from given + * executions to create corresponding {@link RootExceptionHistoryEntry RootExceptionHistoryEntries}. + */ +public class ExceptionHistoryEntryExtractor { + + public RootExceptionHistoryEntry extractLocalFailure( + Map<JobVertexID, ExecutionJobVertex> executionJobVertices, + ExecutionVertexID executionVertexId, + Iterable<ExecutionVertexID> otherAffectedVertices) { + final ExceptionHistoryEntry entry = + extractExceptionHistoryEntry(executionJobVertices, executionVertexId); + + Preconditions.checkArgument( + entry != null, + "The passed ExecutionVertexId {} does not correspond to a failed execution.", + executionVertexId); + + final RootExceptionHistoryEntry root = + RootExceptionHistoryEntry.fromExceptionHistoryEntry(entry); + + for (ExecutionVertexID otherExecutionVertexId : otherAffectedVertices) { + final ExceptionHistoryEntry otherEntry = + extractExceptionHistoryEntry(executionJobVertices, otherExecutionVertexId); + if (otherEntry != null) { + root.add(otherEntry); + } + } + + return root; + } + + public RootExceptionHistoryEntry extractGlobalFailure( + Iterable<ExecutionVertex> executionVertices, Throwable rootCause, long timestamp) { + final RootExceptionHistoryEntry root = + RootExceptionHistoryEntry.fromGlobalFailure(rootCause, timestamp); + + for (ExecutionVertex executionVertex : executionVertices) { + final ExceptionHistoryEntry exceptionHistoryEntry = getFailedExecution(executionVertex); + if (exceptionHistoryEntry != null) { + root.add(exceptionHistoryEntry); + } + } + + return root; + } + + private ExceptionHistoryEntry extractExceptionHistoryEntry( + Map<JobVertexID, ExecutionJobVertex> executionJobVertices, + ExecutionVertexID executionVertexID) { + final ExecutionJobVertex executionJobVertex = + executionJobVertices.get(executionVertexID.getJobVertexId()); + + if (executionJobVertex == null) { + return null; + } + + final ExecutionVertex[] executionVertices = executionJobVertex.getTaskVertices(); + + if (executionVertices.length <= executionVertexID.getSubtaskIndex()) { + return null; Review comment: Why isn't this failing immediately? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractorTest.java ########## @@ -0,0 +1,159 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Range; + +import org.hamcrest.collection.IsIterableContainingInOrder; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertThat; + +/** {@code ExceptionHistoryEntryExtractorTest} tests {@link ExceptionHistoryEntryExtractor}. */ +public class ExceptionHistoryEntryExtractorTest extends TestLogger { + + private ExecutionGraph executionGraph; + + @Before + public void setup() throws JobException, JobExecutionException { + final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); + jobGraph.getVertices().forEach(v -> v.setParallelism(3)); + + executionGraph = + TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + } + + @Test + public void extractLocalFailure() { + final Throwable rootException = new RuntimeException("Expected root failure"); + final ExecutionVertex rootExecutionVertex = extractExecutionVertex(0); + final Range<Long> rootFailureTimeframe = triggerFailure(rootExecutionVertex, rootException); + + final Throwable otherException = new IllegalStateException("Expected other failure"); + final ExecutionVertex otherExecutionVertex = extractExecutionVertex(1); + final Range<Long> otherFailureTimeframe = + triggerFailure(otherExecutionVertex, otherException); + + final ExecutionVertex notFailedExecutionVertex = extractExecutionVertex(2); + + final ExceptionHistoryEntryExtractor testInstance = new ExceptionHistoryEntryExtractor(); + final RootExceptionHistoryEntry actualEntry = + testInstance.extractLocalFailure( + executionGraph.getAllVertices(), + rootExecutionVertex.getID(), + Arrays.asList( + otherExecutionVertex.getID(), notFailedExecutionVertex.getID())); + + assertThat( + actualEntry, + ExceptionHistoryEntryMatcher.matchesFailure( + rootException, + rootFailureTimeframe.lowerEndpoint(), + rootFailureTimeframe.upperEndpoint(), + rootExecutionVertex.getTaskNameWithSubtaskIndex(), + rootExecutionVertex.getCurrentAssignedResourceLocation())); + assertThat( + actualEntry.getConcurrentExceptions(), + IsIterableContainingInOrder.contains( + ExceptionHistoryEntryMatcher.matchesFailure( + otherException, + otherFailureTimeframe.lowerEndpoint(), + otherFailureTimeframe.upperEndpoint(), + otherExecutionVertex.getTaskNameWithSubtaskIndex(), + otherExecutionVertex.getCurrentAssignedResourceLocation()))); + } + + @Test + public void extractGlobalFailure() { + final Throwable otherException0 = new RuntimeException("Expected other failure #0"); Review comment: ```suggestion final Throwable concurrentException0 = new RuntimeException("Expected other failure #0"); ``` ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractorTest.java ########## @@ -0,0 +1,159 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Range; + +import org.hamcrest.collection.IsIterableContainingInOrder; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertThat; + +/** {@code ExceptionHistoryEntryExtractorTest} tests {@link ExceptionHistoryEntryExtractor}. */ +public class ExceptionHistoryEntryExtractorTest extends TestLogger { + + private ExecutionGraph executionGraph; + + @Before + public void setup() throws JobException, JobExecutionException { + final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); + jobGraph.getVertices().forEach(v -> v.setParallelism(3)); + + executionGraph = + TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + } + + @Test + public void extractLocalFailure() { + final Throwable rootException = new RuntimeException("Expected root failure"); + final ExecutionVertex rootExecutionVertex = extractExecutionVertex(0); + final Range<Long> rootFailureTimeframe = triggerFailure(rootExecutionVertex, rootException); + + final Throwable otherException = new IllegalStateException("Expected other failure"); + final ExecutionVertex otherExecutionVertex = extractExecutionVertex(1); + final Range<Long> otherFailureTimeframe = + triggerFailure(otherExecutionVertex, otherException); + + final ExecutionVertex notFailedExecutionVertex = extractExecutionVertex(2); + + final ExceptionHistoryEntryExtractor testInstance = new ExceptionHistoryEntryExtractor(); + final RootExceptionHistoryEntry actualEntry = + testInstance.extractLocalFailure( + executionGraph.getAllVertices(), + rootExecutionVertex.getID(), + Arrays.asList( + otherExecutionVertex.getID(), notFailedExecutionVertex.getID())); + + assertThat( + actualEntry, + ExceptionHistoryEntryMatcher.matchesFailure( + rootException, + rootFailureTimeframe.lowerEndpoint(), + rootFailureTimeframe.upperEndpoint(), + rootExecutionVertex.getTaskNameWithSubtaskIndex(), + rootExecutionVertex.getCurrentAssignedResourceLocation())); + assertThat( + actualEntry.getConcurrentExceptions(), + IsIterableContainingInOrder.contains( + ExceptionHistoryEntryMatcher.matchesFailure( + otherException, + otherFailureTimeframe.lowerEndpoint(), + otherFailureTimeframe.upperEndpoint(), + otherExecutionVertex.getTaskNameWithSubtaskIndex(), + otherExecutionVertex.getCurrentAssignedResourceLocation()))); + } + + @Test + public void extractGlobalFailure() { + final Throwable otherException0 = new RuntimeException("Expected other failure #0"); + final ExecutionVertex otherExecutionVertex0 = extractExecutionVertex(0); + final Range<Long> otherFailureTimeframe0 = + triggerFailure(otherExecutionVertex0, otherException0); + + final Throwable otherException1 = new IllegalStateException("Expected other failure"); + final ExecutionVertex otherExecutionVertex1 = extractExecutionVertex(1); + final Range<Long> otherFailureTimeframe1 = + triggerFailure(otherExecutionVertex1, otherException1); + + final Throwable rootCause = new Exception("Expected root failure"); + final long rootTimestamp = System.currentTimeMillis(); + final ExceptionHistoryEntryExtractor testInstance = new ExceptionHistoryEntryExtractor(); + final RootExceptionHistoryEntry actualEntry = + testInstance.extractGlobalFailure( + executionGraph.getAllExecutionVertices(), rootCause, rootTimestamp); + + assertThat( + actualEntry, + ExceptionHistoryEntryMatcher.matchesGlobalFailure(rootCause, rootTimestamp)); + assertThat( + actualEntry.getConcurrentExceptions(), + IsIterableContainingInOrder.contains( + ExceptionHistoryEntryMatcher.matchesFailure( + otherException0, + otherFailureTimeframe0.lowerEndpoint(), + otherFailureTimeframe0.upperEndpoint(), + otherExecutionVertex0.getTaskNameWithSubtaskIndex(), + otherExecutionVertex0.getCurrentAssignedResourceLocation()), + ExceptionHistoryEntryMatcher.matchesFailure( + otherException1, + otherFailureTimeframe1.lowerEndpoint(), + otherFailureTimeframe1.upperEndpoint(), + otherExecutionVertex1.getTaskNameWithSubtaskIndex(), + otherExecutionVertex1.getCurrentAssignedResourceLocation()))); + } + + private Range<Long> triggerFailure(ExecutionVertex executionVertex, Throwable throwable) { + final long startInMillis = System.currentTimeMillis(); Review comment: shouldn't be necessary; the failure timestamp is the FAILED state timestamp of the vertex ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExceptionHistoryEntryTest.java ########## @@ -24,6 +24,8 @@ import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.IOMetrics; +import org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntry; Review comment: this test should also be moved ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractor.java ########## @@ -0,0 +1,105 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.util.Preconditions; + +import java.util.Map; + +/** + * {@code ExceptionHistoryEntryExtractor} extracts all the necessary information from given + * executions to create corresponding {@link RootExceptionHistoryEntry RootExceptionHistoryEntries}. + */ +public class ExceptionHistoryEntryExtractor { + + public RootExceptionHistoryEntry extractLocalFailure( + Map<JobVertexID, ExecutionJobVertex> executionJobVertices, + ExecutionVertexID executionVertexId, Review comment: ```suggestion ExecutionVertexID failedExecutionVertexId, ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractor.java ########## @@ -0,0 +1,105 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.util.Preconditions; + +import java.util.Map; + +/** + * {@code ExceptionHistoryEntryExtractor} extracts all the necessary information from given + * executions to create corresponding {@link RootExceptionHistoryEntry RootExceptionHistoryEntries}. + */ +public class ExceptionHistoryEntryExtractor { + + public RootExceptionHistoryEntry extractLocalFailure( + Map<JobVertexID, ExecutionJobVertex> executionJobVertices, + ExecutionVertexID executionVertexId, + Iterable<ExecutionVertexID> otherAffectedVertices) { + final ExceptionHistoryEntry entry = + extractExceptionHistoryEntry(executionJobVertices, executionVertexId); + + Preconditions.checkArgument( + entry != null, + "The passed ExecutionVertexId {} does not correspond to a failed execution.", + executionVertexId); + + final RootExceptionHistoryEntry root = + RootExceptionHistoryEntry.fromExceptionHistoryEntry(entry); + + for (ExecutionVertexID otherExecutionVertexId : otherAffectedVertices) { + final ExceptionHistoryEntry otherEntry = + extractExceptionHistoryEntry(executionJobVertices, otherExecutionVertexId); + if (otherEntry != null) { + root.add(otherEntry); + } + } + + return root; + } + + public RootExceptionHistoryEntry extractGlobalFailure( + Iterable<ExecutionVertex> executionVertices, Throwable rootCause, long timestamp) { + final RootExceptionHistoryEntry root = + RootExceptionHistoryEntry.fromGlobalFailure(rootCause, timestamp); + + for (ExecutionVertex executionVertex : executionVertices) { + final ExceptionHistoryEntry exceptionHistoryEntry = getFailedExecution(executionVertex); + if (exceptionHistoryEntry != null) { + root.add(exceptionHistoryEntry); + } + } + + return root; + } + + private ExceptionHistoryEntry extractExceptionHistoryEntry( + Map<JobVertexID, ExecutionJobVertex> executionJobVertices, + ExecutionVertexID executionVertexID) { + final ExecutionJobVertex executionJobVertex = + executionJobVertices.get(executionVertexID.getJobVertexId()); + + if (executionJobVertex == null) { + return null; Review comment: Why isn't this failing immediately? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryExtractorTest.java ########## @@ -0,0 +1,159 @@ +/* + * 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.exceptionhistory; + +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Range; + +import org.hamcrest.collection.IsIterableContainingInOrder; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertThat; + +/** {@code ExceptionHistoryEntryExtractorTest} tests {@link ExceptionHistoryEntryExtractor}. */ +public class ExceptionHistoryEntryExtractorTest extends TestLogger { + + private ExecutionGraph executionGraph; + + @Before + public void setup() throws JobException, JobExecutionException { + final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); + jobGraph.getVertices().forEach(v -> v.setParallelism(3)); + + executionGraph = + TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + } + + @Test + public void extractLocalFailure() { + final Throwable rootException = new RuntimeException("Expected root failure"); + final ExecutionVertex rootExecutionVertex = extractExecutionVertex(0); + final Range<Long> rootFailureTimeframe = triggerFailure(rootExecutionVertex, rootException); + + final Throwable otherException = new IllegalStateException("Expected other failure"); + final ExecutionVertex otherExecutionVertex = extractExecutionVertex(1); + final Range<Long> otherFailureTimeframe = + triggerFailure(otherExecutionVertex, otherException); + + final ExecutionVertex notFailedExecutionVertex = extractExecutionVertex(2); + + final ExceptionHistoryEntryExtractor testInstance = new ExceptionHistoryEntryExtractor(); + final RootExceptionHistoryEntry actualEntry = + testInstance.extractLocalFailure( + executionGraph.getAllVertices(), + rootExecutionVertex.getID(), + Arrays.asList( + otherExecutionVertex.getID(), notFailedExecutionVertex.getID())); + + assertThat( + actualEntry, + ExceptionHistoryEntryMatcher.matchesFailure( + rootException, + rootFailureTimeframe.lowerEndpoint(), + rootFailureTimeframe.upperEndpoint(), + rootExecutionVertex.getTaskNameWithSubtaskIndex(), + rootExecutionVertex.getCurrentAssignedResourceLocation())); + assertThat( + actualEntry.getConcurrentExceptions(), + IsIterableContainingInOrder.contains( + ExceptionHistoryEntryMatcher.matchesFailure( + otherException, + otherFailureTimeframe.lowerEndpoint(), + otherFailureTimeframe.upperEndpoint(), + otherExecutionVertex.getTaskNameWithSubtaskIndex(), + otherExecutionVertex.getCurrentAssignedResourceLocation()))); + } + + @Test + public void extractGlobalFailure() { + final Throwable otherException0 = new RuntimeException("Expected other failure #0"); + final ExecutionVertex otherExecutionVertex0 = extractExecutionVertex(0); + final Range<Long> otherFailureTimeframe0 = + triggerFailure(otherExecutionVertex0, otherException0); + + final Throwable otherException1 = new IllegalStateException("Expected other failure"); Review comment: ```suggestion final Throwable concurrentException1 = new IllegalStateException("Expected other failure"); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org