You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "dmvk (via GitHub)" <gi...@apache.org> on 2023/04/25 16:32:25 UTC

[GitHub] [flink] dmvk commented on a diff in pull request #22467: [FLINK-31888] Introduce interfaces and utils for loading and executing enrichers

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


##########
flink-tests/src/test/java/org/apache/flink/test/plugin/FailureEnricherPluginTest.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.test.plugin;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+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.DefaultPluginManager;
+import org.apache.flink.core.plugin.DirectoryBasedPluginFinder;
+import org.apache.flink.core.plugin.PluginDescriptor;
+import org.apache.flink.core.plugin.PluginFinder;
+import org.apache.flink.core.plugin.PluginManager;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.failure.FailureEnricherUtils;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.test.plugin.jar.failure.TestFailureEnricher;
+import org.apache.flink.test.plugin.jar.failure.TestFailureEnricherFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/** Test for {@link FailureEnricherFactory}. */
+public class FailureEnricherPluginTest extends PluginTestBase {

Review Comment:
   What does this test beyond what DefaultPluginManagerTest already does?



##########
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."
+                                    + " If empty, NO failure enrichers will be started."
+                                    + " If configured, only enrichers whose name matches"
+                                    + " any of the names in the list will be started. ");

Review Comment:
   nit
   ```suggestion
                                       + " any of the names in the list will be started.");
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.api.common.JobID;
+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.metrics.MetricGroup;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.MockedStatic;
+import org.mockito.Mockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.mockStatic;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/** Tests for the {@link FailureEnricherUtils} class. */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({FailureEnricherUtils.class, LoggerFactory.class})
+public class FailureEnricherUtilsTest {

Review Comment:
   new tests should be based on junit5 + assertj



##########
flink-runtime/src/test/java/org/apache/flink/runtime/failure/FailureEnricherUtilsTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.api.common.JobID;
+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.metrics.MetricGroup;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.MockedStatic;
+import org.mockito.Mockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.runtime.failure.FailureEnricherUtils.MERGE_EXCEPTION_MSG;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.mockStatic;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/** Tests for the {@link FailureEnricherUtils} class. */
+@RunWith(PowerMockRunner.class)

Review Comment:
   Mockito shouldn't be used for the new code since it made any refactoring insanely hard because you must fix all mocks individually. We are really on "Testing" implementations instead.
   
   I can walk you through that offline.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org