dmvk commented on code in PR #22467: URL: https://github.com/apache/flink/pull/22467#discussion_r1179264603
########## flink-runtime/src/main/java/org/apache/flink/runtime/failure/FailureEnricherUtils.java: ########## @@ -0,0 +1,227 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricher.Context; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** Utils class for loading and running pluggable failure enrichers. */ +public class FailureEnricherUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FailureEnricherUtils.class); + // regex pattern to split the defined failure enrichers + private static final Pattern enricherListPattern = Pattern.compile("\\s*,\\s*"); + static final String MERGE_EXCEPTION_MSG = + "Trying to merge a label with a duplicate key %s. This is a bug that should be reported," + + " because Flink shouldn't allow registering enrichers with the same output."; + + public static Collection<FailureEnricher> getFailureEnrichers( + final Configuration configuration) { + final PluginManager pluginManager = + PluginUtils.createPluginManagerFromRootFolder(configuration); + return getFailureEnrichers(configuration, pluginManager); + } + + /** + * Returns a set of validated FailureEnrichers for a given configuration, job ID, job name, + * metric group, and plugin manager. + * + * @param configuration the configuration for the job + * @param pluginManager the PluginManager for the job + * @return a collection of validated FailureEnrichers + * @throws IllegalArgumentException if any of the FailureEnrichers are invalid + */ + static Collection<FailureEnricher> getFailureEnrichers( Review Comment: If this method is not meant to be used directly, it should be annotated with `@VisibleForTesting` + the javadocs should be moved to the public one Other option would be making both of them public with javadocs. ########## flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java: ########## @@ -0,0 +1,284 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.TestingPluginManager; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.fail; + +/** Tests for the {@link FailureEnricherUtils} class. */ +@ExtendWith(TestLoggerExtension.class) +class FailureEnricherUtilsTest { + + @Test + public void testGetIncludedFailureEnrichers() { + Configuration conf = new Configuration(); + + // Disabled feature + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, ""); + Set<String> result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(0); + + // Single enricher + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(1); + assertThat(result).contains("enricher1"); + + // Multiple enrichers with spaces + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + + // Multiple enrichers with spaces and empty values + conf.setString( + JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, ,enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + } + + @Test + public void testGetFailureEnrichers() { + final Configuration configuration = new Configuration(); + final Collection<FailureEnricher> emptyEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + + // Empty -> disabled feature + assertThat(emptyEnrichers).hasSize(0); + + // Invalid Name + configuration.set( + JobManagerOptions.FAILURE_ENRICHERS_LIST, FailureEnricherUtilsTest.class.getName()); + final Collection<FailureEnricher> invalidEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + // Excluding failure enricher + assertThat(invalidEnrichers).hasSize(0); + + // Valid Name plus loading + configuration.set(JobManagerOptions.FAILURE_ENRICHERS_LIST, TestEnricher.class.getName()); + final Collection<FailureEnricher> enrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + assertThat(enrichers).hasSize(1); + // verify that the failure enricher was created and returned + assertThat(enrichers.iterator().next()).isInstanceOf(TestEnricher.class); + } + + @Test + public void testGetValidatedEnrichers() { + // create two enrichers with non-overlapping keys + final FailureEnricher firstEnricher = new TestEnricher("key1"); + final FailureEnricher secondEnricher = new TestEnricher("key2"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(firstEnricher); + add(secondEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + + // expect both enrichers to be valid + assertThat(validatedEnrichers).hasSize(2); + assertThat(validatedEnrichers).contains(firstEnricher, secondEnricher); + } + + @Test + public void testValidatedEnrichersWithInvalidEntries() { + // create two enrichers with overlapping keys and a valid one -- must be different classes + final FailureEnricher validEnricher = new TestEnricher("validKey"); + final FailureEnricher firstOverlapEnricher = new AnotherTestEnricher("key1", "key2"); + final FailureEnricher secondOverlapEnricher = new AndAnotherTestEnricher("key2", "key3"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(validEnricher); + add(firstOverlapEnricher); + add(secondOverlapEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + // Only one enricher is valid + assertThat(validatedEnrichers).hasSize(1); + } + + @Test + public void testLabelFutureWithValidEnricher() { + // validate labelFailure by enricher with correct outputKeys + final Throwable cause = new RuntimeException("test exception"); + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher validEnricher = new TestEnricher("enricherKey"); + failureEnrichers.add(validEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + final Map<String, String> labels = result.get(); + assertThat(labels).hasSize(1); + assertThat(labels).containsKey("enricherKey"); + assertThat(labels).containsValue("enricherKeyValue"); + } catch (Exception e) { + fail("This should never happen."); Review Comment: why is this needed? if we throw an exception the test would fail anyway no? ########## flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java: ########## @@ -0,0 +1,284 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.TestingPluginManager; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.fail; + +/** Tests for the {@link FailureEnricherUtils} class. */ +@ExtendWith(TestLoggerExtension.class) +class FailureEnricherUtilsTest { + + @Test + public void testGetIncludedFailureEnrichers() { + Configuration conf = new Configuration(); + + // Disabled feature + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, ""); + Set<String> result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(0); + + // Single enricher + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(1); + assertThat(result).contains("enricher1"); + + // Multiple enrichers with spaces + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + + // Multiple enrichers with spaces and empty values + conf.setString( + JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, ,enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + } + + @Test + public void testGetFailureEnrichers() { + final Configuration configuration = new Configuration(); + final Collection<FailureEnricher> emptyEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + + // Empty -> disabled feature + assertThat(emptyEnrichers).hasSize(0); + + // Invalid Name + configuration.set( + JobManagerOptions.FAILURE_ENRICHERS_LIST, FailureEnricherUtilsTest.class.getName()); + final Collection<FailureEnricher> invalidEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + // Excluding failure enricher + assertThat(invalidEnrichers).hasSize(0); + + // Valid Name plus loading + configuration.set(JobManagerOptions.FAILURE_ENRICHERS_LIST, TestEnricher.class.getName()); + final Collection<FailureEnricher> enrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + assertThat(enrichers).hasSize(1); + // verify that the failure enricher was created and returned + assertThat(enrichers.iterator().next()).isInstanceOf(TestEnricher.class); + } + + @Test + public void testGetValidatedEnrichers() { + // create two enrichers with non-overlapping keys + final FailureEnricher firstEnricher = new TestEnricher("key1"); + final FailureEnricher secondEnricher = new TestEnricher("key2"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(firstEnricher); + add(secondEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + + // expect both enrichers to be valid + assertThat(validatedEnrichers).hasSize(2); + assertThat(validatedEnrichers).contains(firstEnricher, secondEnricher); + } + + @Test + public void testValidatedEnrichersWithInvalidEntries() { + // create two enrichers with overlapping keys and a valid one -- must be different classes + final FailureEnricher validEnricher = new TestEnricher("validKey"); + final FailureEnricher firstOverlapEnricher = new AnotherTestEnricher("key1", "key2"); + final FailureEnricher secondOverlapEnricher = new AndAnotherTestEnricher("key2", "key3"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(validEnricher); + add(firstOverlapEnricher); + add(secondOverlapEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + // Only one enricher is valid + assertThat(validatedEnrichers).hasSize(1); + } + + @Test + public void testLabelFutureWithValidEnricher() { + // validate labelFailure by enricher with correct outputKeys + final Throwable cause = new RuntimeException("test exception"); + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher validEnricher = new TestEnricher("enricherKey"); + failureEnrichers.add(validEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + final Map<String, String> labels = result.get(); + assertThat(labels).hasSize(1); + assertThat(labels).containsKey("enricherKey"); + assertThat(labels).containsValue("enricherKeyValue"); + } catch (Exception e) { + fail("This should never happen."); + } Review Comment: you can use FlinkAssertions to simplify this (applies to multiple places) ```suggestion assertThatFuture(result).eventuallySucceeds().satisfies(labels -> { assertThat(labels).hasSize(1); assertThat(labels).containsKey("enricherKey"); assertThat(labels).containsValue("enricherKeyValue"); }); ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java: ########## @@ -0,0 +1,284 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.TestingPluginManager; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.fail; + +/** Tests for the {@link FailureEnricherUtils} class. */ +@ExtendWith(TestLoggerExtension.class) +class FailureEnricherUtilsTest { + + @Test + public void testGetIncludedFailureEnrichers() { + Configuration conf = new Configuration(); + + // Disabled feature + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, ""); + Set<String> result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(0); + + // Single enricher + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(1); + assertThat(result).contains("enricher1"); + + // Multiple enrichers with spaces + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + + // Multiple enrichers with spaces and empty values + conf.setString( + JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, ,enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + } + + @Test + public void testGetFailureEnrichers() { + final Configuration configuration = new Configuration(); + final Collection<FailureEnricher> emptyEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + + // Empty -> disabled feature + assertThat(emptyEnrichers).hasSize(0); + + // Invalid Name + configuration.set( + JobManagerOptions.FAILURE_ENRICHERS_LIST, FailureEnricherUtilsTest.class.getName()); + final Collection<FailureEnricher> invalidEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + // Excluding failure enricher + assertThat(invalidEnrichers).hasSize(0); + + // Valid Name plus loading + configuration.set(JobManagerOptions.FAILURE_ENRICHERS_LIST, TestEnricher.class.getName()); + final Collection<FailureEnricher> enrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + assertThat(enrichers).hasSize(1); + // verify that the failure enricher was created and returned + assertThat(enrichers.iterator().next()).isInstanceOf(TestEnricher.class); + } + + @Test + public void testGetValidatedEnrichers() { + // create two enrichers with non-overlapping keys + final FailureEnricher firstEnricher = new TestEnricher("key1"); + final FailureEnricher secondEnricher = new TestEnricher("key2"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(firstEnricher); + add(secondEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + + // expect both enrichers to be valid + assertThat(validatedEnrichers).hasSize(2); + assertThat(validatedEnrichers).contains(firstEnricher, secondEnricher); + } + + @Test + public void testValidatedEnrichersWithInvalidEntries() { + // create two enrichers with overlapping keys and a valid one -- must be different classes + final FailureEnricher validEnricher = new TestEnricher("validKey"); + final FailureEnricher firstOverlapEnricher = new AnotherTestEnricher("key1", "key2"); + final FailureEnricher secondOverlapEnricher = new AndAnotherTestEnricher("key2", "key3"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(validEnricher); + add(firstOverlapEnricher); + add(secondOverlapEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + // Only one enricher is valid + assertThat(validatedEnrichers).hasSize(1); + } + + @Test + public void testLabelFutureWithValidEnricher() { + // validate labelFailure by enricher with correct outputKeys + final Throwable cause = new RuntimeException("test exception"); + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher validEnricher = new TestEnricher("enricherKey"); + failureEnrichers.add(validEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + final Map<String, String> labels = result.get(); + assertThat(labels).hasSize(1); + assertThat(labels).containsKey("enricherKey"); + assertThat(labels).containsValue("enricherKeyValue"); + } catch (Exception e) { + fail("This should never happen."); + } + } + + @Test + public void testLabelFailureWithInvalidEnricher() { + // validate labelFailure by enricher with wrong outputKeys + final Throwable cause = new RuntimeException("test exception"); + final String invalidEnricherKey = "invalidKey"; + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher invalidEnricher = + new TestEnricher( + Collections.singletonMap(invalidEnricherKey, "enricherValue"), + "enricherKey"); + failureEnrichers.add(invalidEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + // Ignoring labels + final Map<String, String> labels = result.get(); + assertThat(labels).isEmpty(); + } catch (Exception e) { + fail("This should never happen."); + } + } + + @Test + public void testLabelFailureMergeException() { + // Throwing exception labelFailure when merging duplicate keys + final Throwable cause = new RuntimeException("test failure"); + final FailureEnricher firstEnricher = new TestEnricher("key1", "key2"); + final FailureEnricher secondEnricher = new TestEnricher("key2", "key3"); + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(firstEnricher); + add(secondEnricher); + } + }; + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, enrichers); + + try { + result.get(); + } catch (Exception e) { + assertThat(e).hasMessageContaining(String.format(MERGE_EXCEPTION_MSG, "key2")); + } + } + + /** + * Testing plugin manager for {@link FailureEnricherFactory} utilizing {@link + * TestFailureEnricherFactory}. + * + * @return the testing PluginManager + */ + private PluginManager getPluginManager() { + final Map<Class<?>, Iterator<?>> plugins = new HashMap<>(); + plugins.put( + FailureEnricherFactory.class, + IteratorUtils.singletonIterator(new TestFailureEnricherFactory())); + return new TestingPluginManager(plugins); + } + + /** Factory implementation of {@link TestEnricher} used for plugin load testing. */ + private class TestFailureEnricherFactory implements FailureEnricherFactory { + + @Override + public FailureEnricher createFailureEnricher(Configuration conf) { + return new TestEnricher(); + } + } + + private static class AndAnotherTestEnricher extends TestEnricher { + + public AndAnotherTestEnricher(String... outputKeys) { Review Comment: nit: public constructors don't really make sense with private classes, we usually just use package private for conveniece ```suggestion AndAnotherTestEnricher(String... outputKeys) { ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java: ########## @@ -0,0 +1,284 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.TestingPluginManager; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.fail; + +/** Tests for the {@link FailureEnricherUtils} class. */ +@ExtendWith(TestLoggerExtension.class) +class FailureEnricherUtilsTest { + + @Test + public void testGetIncludedFailureEnrichers() { + Configuration conf = new Configuration(); + + // Disabled feature + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, ""); + Set<String> result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(0); + + // Single enricher + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(1); + assertThat(result).contains("enricher1"); + + // Multiple enrichers with spaces + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + + // Multiple enrichers with spaces and empty values + conf.setString( + JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, ,enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + } + + @Test + public void testGetFailureEnrichers() { + final Configuration configuration = new Configuration(); + final Collection<FailureEnricher> emptyEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + + // Empty -> disabled feature + assertThat(emptyEnrichers).hasSize(0); + + // Invalid Name + configuration.set( + JobManagerOptions.FAILURE_ENRICHERS_LIST, FailureEnricherUtilsTest.class.getName()); + final Collection<FailureEnricher> invalidEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + // Excluding failure enricher + assertThat(invalidEnrichers).hasSize(0); + + // Valid Name plus loading + configuration.set(JobManagerOptions.FAILURE_ENRICHERS_LIST, TestEnricher.class.getName()); + final Collection<FailureEnricher> enrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + assertThat(enrichers).hasSize(1); + // verify that the failure enricher was created and returned + assertThat(enrichers.iterator().next()).isInstanceOf(TestEnricher.class); + } + + @Test + public void testGetValidatedEnrichers() { + // create two enrichers with non-overlapping keys + final FailureEnricher firstEnricher = new TestEnricher("key1"); + final FailureEnricher secondEnricher = new TestEnricher("key2"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(firstEnricher); + add(secondEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + + // expect both enrichers to be valid + assertThat(validatedEnrichers).hasSize(2); + assertThat(validatedEnrichers).contains(firstEnricher, secondEnricher); + } + + @Test + public void testValidatedEnrichersWithInvalidEntries() { + // create two enrichers with overlapping keys and a valid one -- must be different classes + final FailureEnricher validEnricher = new TestEnricher("validKey"); + final FailureEnricher firstOverlapEnricher = new AnotherTestEnricher("key1", "key2"); + final FailureEnricher secondOverlapEnricher = new AndAnotherTestEnricher("key2", "key3"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(validEnricher); + add(firstOverlapEnricher); + add(secondOverlapEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + // Only one enricher is valid + assertThat(validatedEnrichers).hasSize(1); + } + + @Test + public void testLabelFutureWithValidEnricher() { + // validate labelFailure by enricher with correct outputKeys + final Throwable cause = new RuntimeException("test exception"); + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher validEnricher = new TestEnricher("enricherKey"); + failureEnrichers.add(validEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + final Map<String, String> labels = result.get(); + assertThat(labels).hasSize(1); + assertThat(labels).containsKey("enricherKey"); + assertThat(labels).containsValue("enricherKeyValue"); + } catch (Exception e) { + fail("This should never happen."); + } + } + + @Test + public void testLabelFailureWithInvalidEnricher() { + // validate labelFailure by enricher with wrong outputKeys + final Throwable cause = new RuntimeException("test exception"); + final String invalidEnricherKey = "invalidKey"; + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher invalidEnricher = + new TestEnricher( + Collections.singletonMap(invalidEnricherKey, "enricherValue"), + "enricherKey"); + failureEnrichers.add(invalidEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + // Ignoring labels + final Map<String, String> labels = result.get(); + assertThat(labels).isEmpty(); + } catch (Exception e) { + fail("This should never happen."); Review Comment: ? ########## flink-core/src/main/java/org/apache/flink/core/failure/FailureEnricher.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.core.failure; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.MetricGroup; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** + * Failure Enricher enabling custom logic and attaching metadata in the form of labels to each type + * of failure as tracked in the job manager. + */ +@Experimental +public interface FailureEnricher { + + /** + * Method to list all the label Keys the enricher can associate with Values in case of a failure + * {@code processFailure}. Note that Keys must unique and properly defined per enricher + * implementation otherwise will be ignored. + * + * @return the unique label Keys of the FailureEnricher + */ + Set<String> getOutputKeys(); + + /** + * Method to handle a failure as part of the enricher and optionally return a map of KV pairs + * (labels). Note that Values should only be associated with Keys from {@code getOutputKeys} + * method otherwise will be ignored. + * + * @param cause the exception that caused this failure + * @param context the context that includes extra information (e.g., if it was a global failure) + * @return map of KV pairs (labels) associated with the failure + */ + CompletableFuture<Map<String, String>> processFailure( + final Throwable cause, final Context context); + + /** + * An interface used by the {@link FailureEnricher}. Context includes an executor pool for the + * enrichers to run heavy operations, the Classloader used for code gen, and other metadata. + */ + @Experimental + interface Context { + + /** Type of failure. */ + enum FailureType { + /* happened in the scheduler context */ + GLOBAL, + /* happened in the task manager context */ + LOCAL, + /* caused by task manager disconnection/HB timeout */ + TM_LOST Review Comment: We should use javadocs for documenting enum values (you're missing a start in the opening marker) I was thinking a bit more about these values and this is what I've converged to: ``` /** * The failure has occurred in the scheduler context and can't be tracked back to a * particular task. */ GLOBAL, /** The failure has been reported by a particular task. */ TASK, /** * The TaskManager has non-gracefully disconnected from the JobMaster or we have not * received heartbeats for the {@link * org.apache.flink.configuration.HeartbeatManagerOptions#HEARTBEAT_INTERVAL configured * timeout}. */ TASK_MANAGER ``` WDYT? ########## flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java: ########## @@ -0,0 +1,284 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.TestingPluginManager; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.fail; + +/** Tests for the {@link FailureEnricherUtils} class. */ +@ExtendWith(TestLoggerExtension.class) +class FailureEnricherUtilsTest { + + @Test + public void testGetIncludedFailureEnrichers() { + Configuration conf = new Configuration(); + + // Disabled feature + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, ""); + Set<String> result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(0); + + // Single enricher + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(1); + assertThat(result).contains("enricher1"); + + // Multiple enrichers with spaces + conf.setString(JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + + // Multiple enrichers with spaces and empty values + conf.setString( + JobManagerOptions.FAILURE_ENRICHERS_LIST, "enricher1, ,enricher2, enricher3"); + result = FailureEnricherUtils.getIncludedFailureEnrichers(conf); + assertThat(result).hasSize(3); + assertThat(result).contains("enricher1", "enricher2", "enricher3"); + } + + @Test + public void testGetFailureEnrichers() { + final Configuration configuration = new Configuration(); + final Collection<FailureEnricher> emptyEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + + // Empty -> disabled feature + assertThat(emptyEnrichers).hasSize(0); + + // Invalid Name + configuration.set( + JobManagerOptions.FAILURE_ENRICHERS_LIST, FailureEnricherUtilsTest.class.getName()); + final Collection<FailureEnricher> invalidEnrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + // Excluding failure enricher + assertThat(invalidEnrichers).hasSize(0); + + // Valid Name plus loading + configuration.set(JobManagerOptions.FAILURE_ENRICHERS_LIST, TestEnricher.class.getName()); + final Collection<FailureEnricher> enrichers = + FailureEnricherUtils.getFailureEnrichers(configuration, getPluginManager()); + assertThat(enrichers).hasSize(1); + // verify that the failure enricher was created and returned + assertThat(enrichers.iterator().next()).isInstanceOf(TestEnricher.class); + } + + @Test + public void testGetValidatedEnrichers() { + // create two enrichers with non-overlapping keys + final FailureEnricher firstEnricher = new TestEnricher("key1"); + final FailureEnricher secondEnricher = new TestEnricher("key2"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(firstEnricher); + add(secondEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + + // expect both enrichers to be valid + assertThat(validatedEnrichers).hasSize(2); + assertThat(validatedEnrichers).contains(firstEnricher, secondEnricher); + } + + @Test + public void testValidatedEnrichersWithInvalidEntries() { + // create two enrichers with overlapping keys and a valid one -- must be different classes + final FailureEnricher validEnricher = new TestEnricher("validKey"); + final FailureEnricher firstOverlapEnricher = new AnotherTestEnricher("key1", "key2"); + final FailureEnricher secondOverlapEnricher = new AndAnotherTestEnricher("key2", "key3"); + + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(validEnricher); + add(firstOverlapEnricher); + add(secondOverlapEnricher); + } + }; + + final Collection<FailureEnricher> validatedEnrichers = + FailureEnricherUtils.filterInvalidEnrichers(enrichers); + // Only one enricher is valid + assertThat(validatedEnrichers).hasSize(1); + } + + @Test + public void testLabelFutureWithValidEnricher() { + // validate labelFailure by enricher with correct outputKeys + final Throwable cause = new RuntimeException("test exception"); + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher validEnricher = new TestEnricher("enricherKey"); + failureEnrichers.add(validEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + final Map<String, String> labels = result.get(); + assertThat(labels).hasSize(1); + assertThat(labels).containsKey("enricherKey"); + assertThat(labels).containsValue("enricherKeyValue"); + } catch (Exception e) { + fail("This should never happen."); + } + } + + @Test + public void testLabelFailureWithInvalidEnricher() { + // validate labelFailure by enricher with wrong outputKeys + final Throwable cause = new RuntimeException("test exception"); + final String invalidEnricherKey = "invalidKey"; + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + final FailureEnricher invalidEnricher = + new TestEnricher( + Collections.singletonMap(invalidEnricherKey, "enricherValue"), + "enricherKey"); + failureEnrichers.add(invalidEnricher); + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, failureEnrichers); + + try { + // Ignoring labels + final Map<String, String> labels = result.get(); + assertThat(labels).isEmpty(); + } catch (Exception e) { + fail("This should never happen."); + } + } + + @Test + public void testLabelFailureMergeException() { + // Throwing exception labelFailure when merging duplicate keys + final Throwable cause = new RuntimeException("test failure"); + final FailureEnricher firstEnricher = new TestEnricher("key1", "key2"); + final FailureEnricher secondEnricher = new TestEnricher("key2", "key3"); + final Set<FailureEnricher> enrichers = + new HashSet<FailureEnricher>() { + { + add(firstEnricher); + add(secondEnricher); + } + }; + + final CompletableFuture<Map<String, String>> result = + FailureEnricherUtils.labelFailure(cause, null, enrichers); + + try { + result.get(); + } catch (Exception e) { + assertThat(e).hasMessageContaining(String.format(MERGE_EXCEPTION_MSG, "key2")); + } + } + + /** + * Testing plugin manager for {@link FailureEnricherFactory} utilizing {@link + * TestFailureEnricherFactory}. + * + * @return the testing PluginManager + */ + private PluginManager getPluginManager() { + final Map<Class<?>, Iterator<?>> plugins = new HashMap<>(); + plugins.put( + FailureEnricherFactory.class, + IteratorUtils.singletonIterator(new TestFailureEnricherFactory())); + return new TestingPluginManager(plugins); + } + + /** Factory implementation of {@link TestEnricher} used for plugin load testing. */ + private class TestFailureEnricherFactory implements FailureEnricherFactory { Review Comment: ```suggestion private static PluginManager createPluginManager() { final Map<Class<?>, Iterator<?>> plugins = new HashMap<>(); plugins.put( FailureEnricherFactory.class, IteratorUtils.singletonIterator(new TestFailureEnricherFactory())); return new TestingPluginManager(plugins); } /** Factory implementation of {@link TestEnricher} used for plugin load testing. */ private static class TestFailureEnricherFactory implements FailureEnricherFactory { ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/failure/FailureEnricherUtils.java: ########## @@ -0,0 +1,227 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricher.Context; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** Utils class for loading and running pluggable failure enrichers. */ +public class FailureEnricherUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FailureEnricherUtils.class); + // regex pattern to split the defined failure enrichers + private static final Pattern enricherListPattern = Pattern.compile("\\s*,\\s*"); + static final String MERGE_EXCEPTION_MSG = + "Trying to merge a label with a duplicate key %s. This is a bug that should be reported," + + " because Flink shouldn't allow registering enrichers with the same output."; + + public static Collection<FailureEnricher> getFailureEnrichers( + final Configuration configuration) { + final PluginManager pluginManager = + PluginUtils.createPluginManagerFromRootFolder(configuration); + return getFailureEnrichers(configuration, pluginManager); + } + + /** + * Returns a set of validated FailureEnrichers for a given configuration, job ID, job name, + * metric group, and plugin manager. + * + * @param configuration the configuration for the job + * @param pluginManager the PluginManager for the job + * @return a collection of validated FailureEnrichers + * @throws IllegalArgumentException if any of the FailureEnrichers are invalid + */ + static Collection<FailureEnricher> getFailureEnrichers( + final Configuration configuration, final PluginManager pluginManager) { + Set<String> includedEnrichers = getIncludedFailureEnrichers(configuration); + // When empty, NO enrichers will be started. + if (includedEnrichers.isEmpty()) { + return Collections.emptySet(); + } + final Iterator<FailureEnricherFactory> factoryIterator = + pluginManager.load(FailureEnricherFactory.class); + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + while (factoryIterator.hasNext()) { + try { + final FailureEnricherFactory failureEnricherFactory = factoryIterator.next(); + final FailureEnricher failureEnricher = + failureEnricherFactory.createFailureEnricher(configuration); + if (includedEnrichers.contains(failureEnricher.getClass().getName())) { + failureEnrichers.add(failureEnricher); + LOG.debug( + "Found failure enricher {} at {} ", + failureEnricherFactory.getClass().getName(), + new File( + failureEnricher + .getClass() + .getProtectionDomain() + .getCodeSource() + .getLocation() + .toURI()) + .getCanonicalPath()); + } else { + LOG.info( + "Excluding failure enricher {}, not configured in enricher list ({}).", + failureEnricherFactory.getClass().getName(), + includedEnrichers); + } + } catch (Exception e) { + LOG.warn("Error while loading failure enricher factory.", e); Review Comment: I'd personally throw an exception here because this violates the cluster setup that the user has asked for. Also, since this happens during JM startup, it should be fine to fail fast. ########## flink-runtime/src/main/java/org/apache/flink/runtime/failure/FailureEnricherUtils.java: ########## @@ -0,0 +1,227 @@ +/* + * 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.failure; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.failure.FailureEnricher; +import org.apache.flink.core.failure.FailureEnricher.Context; +import org.apache.flink.core.failure.FailureEnricherFactory; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** Utils class for loading and running pluggable failure enrichers. */ +public class FailureEnricherUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FailureEnricherUtils.class); + // regex pattern to split the defined failure enrichers + private static final Pattern enricherListPattern = Pattern.compile("\\s*,\\s*"); + static final String MERGE_EXCEPTION_MSG = + "Trying to merge a label with a duplicate key %s. This is a bug that should be reported," + + " because Flink shouldn't allow registering enrichers with the same output."; + + public static Collection<FailureEnricher> getFailureEnrichers( + final Configuration configuration) { + final PluginManager pluginManager = + PluginUtils.createPluginManagerFromRootFolder(configuration); + return getFailureEnrichers(configuration, pluginManager); + } + + /** + * Returns a set of validated FailureEnrichers for a given configuration, job ID, job name, + * metric group, and plugin manager. + * + * @param configuration the configuration for the job + * @param pluginManager the PluginManager for the job + * @return a collection of validated FailureEnrichers + * @throws IllegalArgumentException if any of the FailureEnrichers are invalid + */ + static Collection<FailureEnricher> getFailureEnrichers( + final Configuration configuration, final PluginManager pluginManager) { + Set<String> includedEnrichers = getIncludedFailureEnrichers(configuration); + // When empty, NO enrichers will be started. + if (includedEnrichers.isEmpty()) { + return Collections.emptySet(); + } + final Iterator<FailureEnricherFactory> factoryIterator = + pluginManager.load(FailureEnricherFactory.class); + final Set<FailureEnricher> failureEnrichers = new HashSet<>(); + while (factoryIterator.hasNext()) { + try { + final FailureEnricherFactory failureEnricherFactory = factoryIterator.next(); + final FailureEnricher failureEnricher = + failureEnricherFactory.createFailureEnricher(configuration); + if (includedEnrichers.contains(failureEnricher.getClass().getName())) { + failureEnrichers.add(failureEnricher); + LOG.debug( Review Comment: This should be logged on the INFO level because that's the significant bit. We don't really care about the ones that were skipped (we can log those on DEBUG level) -- 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. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org