pgaref commented on code in PR #22467:
URL: https://github.com/apache/flink/pull/22467#discussion_r1179537961


##########
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 agree, but lets leave it as is for now and add the exception when we 
decide where we are call the factory (JM/Dispatcher/ClusterEntryPoint)



-- 
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

Reply via email to