Copilot commented on code in PR #10095: URL: https://github.com/apache/ozone/pull/10095#discussion_r3121210360
########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/eventlistener/OMEventListenerPluginManager.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.hadoop.ozone.om.eventlistener; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is a manager for plugins which implement OMEventListener which + * manages the lifecycle of constructing starting/stopping configured + * plugins. + */ +public class OMEventListenerPluginManager { + public static final Logger LOG = LoggerFactory.getLogger(OMEventListenerPluginManager.class); + + public static final String PLUGIN_DEST_BASE = "ozone.om.plugin.destination"; + + private final List<OMEventListener> plugins; + + public OMEventListenerPluginManager(OzoneManager ozoneManager, OzoneConfiguration conf) { + this.plugins = loadAll(ozoneManager, conf); + } + + public List<OMEventListener> getLoaded() { + return plugins; + } + + public void startAll() { + for (OMEventListener plugin : plugins) { + plugin.start(); + } + } + + public void shutdownAll() { + for (OMEventListener plugin : plugins) { + plugin.shutdown(); + } + } + + // Configuration is based on ranger plugins + // + // For example, a plugin named FooPlugin would be configured via + // OzoneConfiguration properties as follows: + // + // conf.set("ozone.om.plugin.destination.foo", "enabled"); + // conf.set("ozone.om.plugin.destination.foo.classname", "org.apache.hadoop.ozone.om.eventlistener.FooPlugin"); + // + static List<OMEventListener> loadAll(OzoneManager ozoneManager, OzoneConfiguration conf) { + List<OMEventListener> plugins = new ArrayList<>(); + + Map<String, String> props = conf.getPropsMatchPrefixAndTrimPrefix(PLUGIN_DEST_BASE); + List<String> destNameList = new ArrayList<>(); Review Comment: `getPropsMatchPrefixAndTrimPrefix` is called with `PLUGIN_DEST_BASE` without a trailing dot, which means the trimmed keys will start with a leading `.` (eg `.foo`) and the match can also include unintended keys like `ozone.om.plugin.destinationX...`. This makes later concatenation (`PLUGIN_DEST_BASE + destName + ".classname"`) fragile. Consider using a prefix with the delimiter (eg `PLUGIN_DEST_BASE + "."`) and keeping `destName` normalized (no leading dot). ########## hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/eventlistener/TestOMEventListenerPluginManager.java: ########## @@ -0,0 +1,119 @@ +/* + * 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.hadoop.ozone.om.eventlistener; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +/** + * Tests {@link OMEventListenerPluginManager}. + */ +@ExtendWith(MockitoExtension.class) +public class TestOMEventListenerPluginManager { + + @Mock + private OzoneManager ozoneManager; + + static List<String> getLoadedPlugins(OMEventListenerPluginManager pluginManager) { + List<String> loadedClasses = new ArrayList<>(); + for (OMEventListener plugin : pluginManager.getLoaded()) { + loadedClasses.add(plugin.getClass().getName()); + } + + // normalize + Collections.sort(loadedClasses); + + return loadedClasses; + } + + private static class BrokenFooPlugin { + + } + + @Test + public void testLoadSinglePlugin() throws InterruptedException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.set("ozone.om.plugin.destination.foo", "enabled"); Review Comment: These tests declare `throws InterruptedException`, but nothing in the test body blocks or interrupts. Removing the unused checked exception would simplify the test signatures. ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/eventlistener/OMEventListenerPluginManager.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.hadoop.ozone.om.eventlistener; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is a manager for plugins which implement OMEventListener which + * manages the lifecycle of constructing starting/stopping configured + * plugins. + */ +public class OMEventListenerPluginManager { + public static final Logger LOG = LoggerFactory.getLogger(OMEventListenerPluginManager.class); + + public static final String PLUGIN_DEST_BASE = "ozone.om.plugin.destination"; + + private final List<OMEventListener> plugins; + + public OMEventListenerPluginManager(OzoneManager ozoneManager, OzoneConfiguration conf) { + this.plugins = loadAll(ozoneManager, conf); + } + + public List<OMEventListener> getLoaded() { + return plugins; + } + + public void startAll() { + for (OMEventListener plugin : plugins) { + plugin.start(); + } + } + + public void shutdownAll() { + for (OMEventListener plugin : plugins) { + plugin.shutdown(); + } + } + + // Configuration is based on ranger plugins + // + // For example, a plugin named FooPlugin would be configured via + // OzoneConfiguration properties as follows: + // + // conf.set("ozone.om.plugin.destination.foo", "enabled"); + // conf.set("ozone.om.plugin.destination.foo.classname", "org.apache.hadoop.ozone.om.eventlistener.FooPlugin"); + // + static List<OMEventListener> loadAll(OzoneManager ozoneManager, OzoneConfiguration conf) { + List<OMEventListener> plugins = new ArrayList<>(); + + Map<String, String> props = conf.getPropsMatchPrefixAndTrimPrefix(PLUGIN_DEST_BASE); + List<String> destNameList = new ArrayList<>(); + for (Map.Entry<String, String> entry : props.entrySet()) { + String destName = entry.getKey(); + String value = entry.getValue(); + LOG.info("Found event listener plugin with name={} and value={}", destName, value); + + if (value.equalsIgnoreCase("enable") || value.equalsIgnoreCase("enabled") || value.equalsIgnoreCase("true")) { + destNameList.add(destName); + LOG.info("Event listener plugin {}{} is set to {}", PLUGIN_DEST_BASE, destName, value); + } + } + + OMEventListenerPluginContext pluginContext = new OMEventListenerPluginContextImpl(ozoneManager); + + for (String destName : destNameList) { + try { + Class<? extends OMEventListener> cls = resolvePluginClass(conf, destName); + LOG.info("Event listener plugin class is {}", cls); + + OMEventListener impl = cls.newInstance(); + impl.initialize(conf, pluginContext); + + plugins.add(impl); + } catch (Exception ex) { Review Comment: `Class#newInstance()` is deprecated and can fail for classes without a public no-arg constructor; it also provides less useful exception context. Prefer `cls.getDeclaredConstructor().newInstance()` and catch `ReflectiveOperationException` (or the specific exceptions) so failures are logged with clear cause. ```suggestion OMEventListener impl = cls.getDeclaredConstructor().newInstance(); impl.initialize(conf, pluginContext); plugins.add(impl); } catch (ReflectiveOperationException ex) { ``` ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/eventlistener/OMEventListenerPluginManager.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.hadoop.ozone.om.eventlistener; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is a manager for plugins which implement OMEventListener which + * manages the lifecycle of constructing starting/stopping configured + * plugins. + */ +public class OMEventListenerPluginManager { + public static final Logger LOG = LoggerFactory.getLogger(OMEventListenerPluginManager.class); + + public static final String PLUGIN_DEST_BASE = "ozone.om.plugin.destination"; + + private final List<OMEventListener> plugins; + + public OMEventListenerPluginManager(OzoneManager ozoneManager, OzoneConfiguration conf) { + this.plugins = loadAll(ozoneManager, conf); + } + + public List<OMEventListener> getLoaded() { + return plugins; + } + + public void startAll() { + for (OMEventListener plugin : plugins) { + plugin.start(); + } + } + + public void shutdownAll() { + for (OMEventListener plugin : plugins) { + plugin.shutdown(); Review Comment: If any plugin throws from `shutdown()`, this will abort shutting down subsequent plugins. Consider handling exceptions per plugin and continuing shutdown so a single failure doesn't block cleanup of the rest. ```suggestion try { plugin.shutdown(); } catch (Exception ex) { LOG.error("Failed to shut down event listener plugin {}", plugin, ex); } ``` ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/eventlistener/OMEventListenerPluginManager.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.hadoop.ozone.om.eventlistener; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is a manager for plugins which implement OMEventListener which + * manages the lifecycle of constructing starting/stopping configured + * plugins. + */ +public class OMEventListenerPluginManager { + public static final Logger LOG = LoggerFactory.getLogger(OMEventListenerPluginManager.class); + + public static final String PLUGIN_DEST_BASE = "ozone.om.plugin.destination"; + + private final List<OMEventListener> plugins; + + public OMEventListenerPluginManager(OzoneManager ozoneManager, OzoneConfiguration conf) { + this.plugins = loadAll(ozoneManager, conf); + } + + public List<OMEventListener> getLoaded() { + return plugins; + } + + public void startAll() { + for (OMEventListener plugin : plugins) { + plugin.start(); Review Comment: If any plugin throws from `start()`, this will abort starting subsequent plugins. Consider wrapping each `plugin.start()` in its own try/catch so one bad plugin doesn't prevent others from starting, and log the plugin class/name on failure. ```suggestion try { plugin.start(); } catch (Exception ex) { LOG.error("Failed to start event listener plugin {}", plugin.getClass().getName(), ex); } ``` ########## hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/eventlistener/OMEventListenerPluginManager.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.hadoop.ozone.om.eventlistener; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is a manager for plugins which implement OMEventListener which + * manages the lifecycle of constructing starting/stopping configured + * plugins. + */ +public class OMEventListenerPluginManager { + public static final Logger LOG = LoggerFactory.getLogger(OMEventListenerPluginManager.class); + + public static final String PLUGIN_DEST_BASE = "ozone.om.plugin.destination"; + + private final List<OMEventListener> plugins; + + public OMEventListenerPluginManager(OzoneManager ozoneManager, OzoneConfiguration conf) { + this.plugins = loadAll(ozoneManager, conf); + } + + public List<OMEventListener> getLoaded() { + return plugins; + } + + public void startAll() { + for (OMEventListener plugin : plugins) { + plugin.start(); + } + } + + public void shutdownAll() { + for (OMEventListener plugin : plugins) { + plugin.shutdown(); + } + } + + // Configuration is based on ranger plugins + // + // For example, a plugin named FooPlugin would be configured via + // OzoneConfiguration properties as follows: + // + // conf.set("ozone.om.plugin.destination.foo", "enabled"); + // conf.set("ozone.om.plugin.destination.foo.classname", "org.apache.hadoop.ozone.om.eventlistener.FooPlugin"); + // + static List<OMEventListener> loadAll(OzoneManager ozoneManager, OzoneConfiguration conf) { + List<OMEventListener> plugins = new ArrayList<>(); + + Map<String, String> props = conf.getPropsMatchPrefixAndTrimPrefix(PLUGIN_DEST_BASE); + List<String> destNameList = new ArrayList<>(); + for (Map.Entry<String, String> entry : props.entrySet()) { + String destName = entry.getKey(); + String value = entry.getValue(); + LOG.info("Found event listener plugin with name={} and value={}", destName, value); + + if (value.equalsIgnoreCase("enable") || value.equalsIgnoreCase("enabled") || value.equalsIgnoreCase("true")) { + destNameList.add(destName); + LOG.info("Event listener plugin {}{} is set to {}", PLUGIN_DEST_BASE, destName, value); + } + } + + OMEventListenerPluginContext pluginContext = new OMEventListenerPluginContextImpl(ozoneManager); + + for (String destName : destNameList) { + try { + Class<? extends OMEventListener> cls = resolvePluginClass(conf, destName); + LOG.info("Event listener plugin class is {}", cls); + + OMEventListener impl = cls.newInstance(); + impl.initialize(conf, pluginContext); + + plugins.add(impl); + } catch (Exception ex) { + LOG.error("Can't make instance of event listener plugin {}{}", PLUGIN_DEST_BASE, destName, ex); + } + } + + return plugins; + } + + private static Class<? extends OMEventListener> resolvePluginClass(OzoneConfiguration conf, + String destName) { + String classnameProp = PLUGIN_DEST_BASE + destName + ".classname"; + LOG.info("Gettting classname for {} with propety {}", destName, classnameProp); Review Comment: Typo in the log message: "Gettting" / "propety". Fixing this will make logs easier to search/understand. ```suggestion LOG.info("Getting classname for {} with property {}", destName, classnameProp); ``` ########## hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/eventlistener/TestOMEventListenerPluginManager.java: ########## @@ -0,0 +1,119 @@ +/* + * 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.hadoop.ozone.om.eventlistener; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +/** + * Tests {@link OMEventListenerPluginManager}. + */ +@ExtendWith(MockitoExtension.class) +public class TestOMEventListenerPluginManager { + + @Mock + private OzoneManager ozoneManager; + + static List<String> getLoadedPlugins(OMEventListenerPluginManager pluginManager) { + List<String> loadedClasses = new ArrayList<>(); + for (OMEventListener plugin : pluginManager.getLoaded()) { + loadedClasses.add(plugin.getClass().getName()); + } + + // normalize + Collections.sort(loadedClasses); + + return loadedClasses; + } + + private static class BrokenFooPlugin { + + } + + @Test + public void testLoadSinglePlugin() throws InterruptedException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.set("ozone.om.plugin.destination.foo", "enabled"); + conf.set("ozone.om.plugin.destination.foo.classname", "org.apache.hadoop.ozone.om.eventlistener.FooPlugin"); + + OMEventListenerPluginManager pluginManager = new OMEventListenerPluginManager(ozoneManager, conf); + + Assertions.assertEquals(Arrays.asList("org.apache.hadoop.ozone.om.eventlistener.FooPlugin"), + getLoadedPlugins(pluginManager)); + } + + @Test + public void testLoadMultiplePlugins() throws InterruptedException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.set("ozone.om.plugin.destination.foo", "enabled"); + conf.set("ozone.om.plugin.destination.foo.classname", "org.apache.hadoop.ozone.om.eventlistener.FooPlugin"); + conf.set("ozone.om.plugin.destination.bar", "enabled"); + conf.set("ozone.om.plugin.destination.bar.classname", "org.apache.hadoop.ozone.om.eventlistener.BarPlugin"); + + OMEventListenerPluginManager pluginManager = new OMEventListenerPluginManager(ozoneManager, conf); + + Assertions.assertEquals(Arrays.asList("org.apache.hadoop.ozone.om.eventlistener.BarPlugin", + "org.apache.hadoop.ozone.om.eventlistener.FooPlugin"), + + getLoadedPlugins(pluginManager)); + } + + @Test + public void testPluginMissingClassname() throws InterruptedException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.set("ozone.om.plugin.destination.foo", "enabled"); + + OMEventListenerPluginManager pluginManager = new OMEventListenerPluginManager(ozoneManager, conf); + + Assertions.assertEquals(Arrays.asList(), + getLoadedPlugins(pluginManager)); + } + + @Test + public void testPluginClassDoesNotExist() throws InterruptedException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.set("ozone.om.plugin.destination.foo", "enabled"); + conf.set("ozone.om.plugin.destination.foo.classname", "org.apache.hadoop.ozone.om.eventlistener.NotExistingPlugin"); + + OMEventListenerPluginManager pluginManager = new OMEventListenerPluginManager(ozoneManager, conf); + + Assertions.assertEquals(Arrays.asList(), + getLoadedPlugins(pluginManager)); + } + + @Test + public void testPluginClassDoesNotImplementInterface() throws InterruptedException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.set("ozone.om.plugin.destination.foo", "enabled"); + conf.set("ozone.om.plugin.destination.foo.classname", "org.apache.hadoop.ozone.om.eventlistener.BrokenFooPlugin"); Review Comment: `testPluginClassDoesNotImplementInterface` configures classname `org.apache.hadoop.ozone.om.eventlistener.BrokenFooPlugin`, but `BrokenFooPlugin` is declared as a private static nested class in this test, so its runtime name is `TestOMEventListenerPluginManager$BrokenFooPlugin`. As written, this test is effectively exercising the "class does not exist" path rather than the "does not implement interface" path. Consider using `BrokenFooPlugin.class.getName()` or moving `BrokenFooPlugin` to its own top-level test class in the same package. ```suggestion conf.set("ozone.om.plugin.destination.foo.classname", BrokenFooPlugin.class.getName()); ``` -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
