nateab commented on code in PR #22467:
URL: https://github.com/apache/flink/pull/22467#discussion_r1180829385
##########
flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java:
##########
@@ -263,6 +263,30 @@ public class JobManagerOptions {
.withDescription(
"The maximum number of historical execution
attempts kept in history.");
+ /**
+ * Flag indicating whether JobManager should load available Failure
Enricher plugins at startup.
+ * An optional list of Failure Enricher names. If empty, NO enrichers will
be started. If
+ * configured, only enrichers whose name (as returned by class.getName())
matches any of the
+ * names in the list will be started.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * jobmanager.failure-enrichers =
org.apache.flink.test.plugin.jar.failure.TypeFailureEnricher,
org.apache.flink.runtime.failure.FailureEnricherUtilsTest$TestEnricher
+ *
+ * }</pre>
+ */
+ @Documentation.Section(Documentation.Sections.ALL_JOB_MANAGER)
+ public static final ConfigOption<String> FAILURE_ENRICHERS_LIST =
+ key("jobmanager.failure-enrichers")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ "An optional list of failure enricher names."
Review Comment:
should we specify that it is a comma separated list?
##########
flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.core.testutils.FlinkAssertions.assertThatFuture;
+import static
org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** 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");
+ }
Review Comment:
maybe add a test case for malformed lists such as using a different
delimiter, like a period instead of a comma
##########
flink-core/src/main/java/org/apache/flink/core/failure/FailureEnricher.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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
Review Comment:
should we clarify here that the labels are key value pairs?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]