You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/03/22 13:22:35 UTC

[GitHub] [nifi] exceptionfactory commented on a change in pull request #4898: NIFI-8325: Improve SNMP processors

exceptionfactory commented on a change in pull request #4898:
URL: https://github.com/apache/nifi/pull/4898#discussion_r598682741



##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/context/SNMPContext.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.nifi.snmp.context;
+
+import org.apache.nifi.snmp.configuration.BasicConfiguration;
+import org.apache.nifi.snmp.configuration.SecurityConfiguration;
+import org.apache.nifi.snmp.exception.AgentSecurityConfigurationException;
+import org.apache.nifi.snmp.utils.SNMPUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.snmp4j.AbstractTarget;
+import org.snmp4j.CommunityTarget;
+import org.snmp4j.Snmp;
+import org.snmp4j.UserTarget;
+import org.snmp4j.mp.SnmpConstants;
+import org.snmp4j.security.SecurityLevel;
+import org.snmp4j.security.UsmUser;
+import org.snmp4j.smi.OctetString;
+import org.snmp4j.smi.UdpAddress;
+import org.snmp4j.transport.DefaultUdpTransportMapping;
+
+import java.io.IOException;
+
+public class SNMPContext {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(SNMPContext.class);
+
+    private Snmp snmp;
+    private AbstractTarget target;
+
+    public static SNMPContext newInstance() {
+        return new SNMPContext();
+    }
+
+    public void init(final BasicConfiguration basicConfiguration, final SecurityConfiguration securityConfiguration) {
+        initSnmp(basicConfiguration);
+
+        final String snmpVersion = securityConfiguration.getVersion();
+
+        final int version = SNMPUtils.getSnmpVersion(snmpVersion);
+
+        if (version == SnmpConstants.version3) {
+            createUserTarget(basicConfiguration, securityConfiguration, snmp, version);
+        } else {
+            target = createCommunityTarget(basicConfiguration, securityConfiguration, version);
+        }
+    }
+
+    public void close() {
+        try {
+            snmp.close();
+        } catch (IOException e) {
+            LOGGER.error("Could not close SNMP session.");
+        }
+
+    }
+
+    private CommunityTarget createCommunityTarget(BasicConfiguration basicConfiguration, SecurityConfiguration securityConfiguration, int version) {
+        CommunityTarget communityTarget = new CommunityTarget();
+        setupTargetBasicProperties(communityTarget, basicConfiguration, version);
+        String community = securityConfiguration.getCommunityString();
+        if (community != null) {
+            communityTarget.setCommunity(new OctetString(community));
+        }
+        return communityTarget;
+    }
+
+    private void createUserTarget(BasicConfiguration basicConfiguration, SecurityConfiguration securityConfiguration, Snmp snmp, int version) {
+        final String username = securityConfiguration.getSecurityName();
+        final String authProtocol = securityConfiguration.getAuthProtocol();
+        final String authPassword = securityConfiguration.getAuthPassword();
+        final String privacyProtocol = securityConfiguration.getPrivacyProtocol();
+        final String privacyPassword = securityConfiguration.getPrivacyPassword();
+        final OctetString authPasswordOctet = authPassword != null ? new OctetString(authPassword) : null;
+        final OctetString privacyPasswordOctet = privacyPassword != null ? new OctetString(privacyPassword) : null;
+
+        if (snmp.getUSM() == null) {
+            throw new AgentSecurityConfigurationException("No security model has been configured in agent.");
+        }
+
+        // Add user information.
+        snmp.getUSM().addUser(
+                new OctetString(username),
+                new UsmUser(new OctetString(username), SNMPUtils.getAuth(authProtocol), authPasswordOctet,
+                        SNMPUtils.getPriv(privacyProtocol), privacyPasswordOctet));
+
+        target = new UserTarget();
+        setupTargetBasicProperties(target, basicConfiguration, version);
+        int securityLevel = SecurityLevel.valueOf(securityConfiguration.getSecurityLevel()).getSnmpValue();
+        target.setSecurityLevel(securityLevel);
+
+        final String securityName = securityConfiguration.getSecurityName();
+        if (securityName != null) {
+            target.setSecurityName(new OctetString(securityName));
+        }
+    }
+
+    private void initSnmp(final BasicConfiguration basicConfiguration) {
+        int clientPort = basicConfiguration.getClientPort();
+        try {
+            snmp = new Snmp(new DefaultUdpTransportMapping(new UdpAddress("0.0.0.0/" + clientPort)));

Review comment:
       Is there a reason for hard-coding `0.0.0.0` here instead of having a configurable property on `BasicConfiguration`?

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/configuration/SecurityConfigurationBuilder.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.nifi.snmp.configuration;
+
+public class SecurityConfigurationBuilder {
+    private String version;
+    private String authProtocol;
+    private String authPassword;
+    private String privacyProtocol;
+    private String privacyPassword;
+    private String securityName;
+    private String securityLevel;
+    private String communityString;
+
+    public SecurityConfigurationBuilder setVersion(String version) {
+        this.version = version;
+        return this;
+    }
+
+    public SecurityConfigurationBuilder setAuthProtocol(String authProtocol) {
+        this.authProtocol = authProtocol;
+        return this;
+    }
+
+    public SecurityConfigurationBuilder setAuthPassword(String authPassword) {
+        this.authPassword = authPassword;
+        return this;
+    }
+
+    public SecurityConfigurationBuilder setPrivacyProtocol(String privacyProtocol) {
+        this.privacyProtocol = privacyProtocol;
+        return this;
+    }
+
+    public SecurityConfigurationBuilder setPrivacyPassword(String privacyPassword) {
+        this.privacyPassword = privacyPassword;
+        return this;
+    }
+
+    public SecurityConfigurationBuilder setSecurityName(String securityName) {
+        this.securityName = securityName;
+        return this;
+    }
+
+    public SecurityConfigurationBuilder setSecurityLevel(String securityLevel) {
+        this.securityLevel = securityLevel;
+        return this;
+    }
+
+    public SecurityConfigurationBuilder setCommunityString(String communityString) {
+        this.communityString = communityString;
+        return this;
+    }
+
+    public SecurityConfiguration createSecurityConfiguration() {
+        return new SecurityConfiguration(version, authProtocol, authPassword, privacyProtocol, privacyPassword, securityName, securityLevel, communityString);

Review comment:
       Is it acceptable for all of these values to be `null`?  If any of them are required, it would be helpful to introduce some additional checking prior to constructing the `SecurityConfiguration`.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/context/SNMPContext.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.nifi.snmp.context;
+
+import org.apache.nifi.snmp.configuration.BasicConfiguration;
+import org.apache.nifi.snmp.configuration.SecurityConfiguration;
+import org.apache.nifi.snmp.exception.AgentSecurityConfigurationException;
+import org.apache.nifi.snmp.utils.SNMPUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.snmp4j.AbstractTarget;
+import org.snmp4j.CommunityTarget;
+import org.snmp4j.Snmp;
+import org.snmp4j.UserTarget;
+import org.snmp4j.mp.SnmpConstants;
+import org.snmp4j.security.SecurityLevel;
+import org.snmp4j.security.UsmUser;
+import org.snmp4j.smi.OctetString;
+import org.snmp4j.smi.UdpAddress;
+import org.snmp4j.transport.DefaultUdpTransportMapping;
+
+import java.io.IOException;
+
+public class SNMPContext {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(SNMPContext.class);
+
+    private Snmp snmp;
+    private AbstractTarget target;
+
+    public static SNMPContext newInstance() {
+        return new SNMPContext();
+    }
+
+    public void init(final BasicConfiguration basicConfiguration, final SecurityConfiguration securityConfiguration) {
+        initSnmp(basicConfiguration);
+
+        final String snmpVersion = securityConfiguration.getVersion();
+
+        final int version = SNMPUtils.getSnmpVersion(snmpVersion);
+
+        if (version == SnmpConstants.version3) {
+            createUserTarget(basicConfiguration, securityConfiguration, snmp, version);
+        } else {
+            target = createCommunityTarget(basicConfiguration, securityConfiguration, version);
+        }
+    }
+
+    public void close() {
+        try {
+            snmp.close();
+        } catch (IOException e) {
+            LOGGER.error("Could not close SNMP session.");

Review comment:
       It would be helpful to pass the exception to the logger for troubleshooting.
   ```suggestion
               LOGGER.error("Could not close SNMP session", e);
   ```

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/processors/AbstractSNMPProcessor.java
##########
@@ -16,66 +16,69 @@
  */
 package org.apache.nifi.snmp.processors;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.snmp4j.AbstractTarget;
-import org.snmp4j.CommunityTarget;
-import org.snmp4j.Snmp;
-import org.snmp4j.TransportMapping;
-import org.snmp4j.UserTarget;
-import org.snmp4j.mp.MPv3;
-import org.snmp4j.mp.SnmpConstants;
-import org.snmp4j.security.SecurityModels;
-import org.snmp4j.security.SecurityProtocols;
-import org.snmp4j.security.USM;
-import org.snmp4j.security.UsmUser;
-import org.snmp4j.smi.OctetString;
-import org.snmp4j.smi.UdpAddress;
-import org.snmp4j.transport.DefaultUdpTransportMapping;
+import org.apache.nifi.snmp.configuration.BasicConfiguration;
+import org.apache.nifi.snmp.configuration.SecurityConfiguration;
+import org.apache.nifi.snmp.configuration.SecurityConfigurationBuilder;
+import org.apache.nifi.snmp.context.SNMPContext;
+import org.apache.nifi.snmp.logging.Slf4jLogFactory;
+import org.apache.nifi.snmp.validators.OIDValidator;
+import org.snmp4j.log.LogFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
 
 /**
- * Base processor that uses SNMP4J client API
+ * Base processor that uses SNMP4J client API.
  * (http://www.snmp4j.org/)
- *
- * @param <T> the type of {@link SNMPWorker}. Please see {@link SNMPSetter}
- *            and {@link SNMPGetter}
  */
-abstract class AbstractSNMPProcessor<T extends SNMPWorker> extends AbstractProcessor {
+abstract class AbstractSNMPProcessor extends AbstractProcessor {
+
+    static {
+        LogFactory.setLogFactory(new Slf4jLogFactory());
+    }
+
+    // Property to define the host of the SNMP agent.
+    public static final PropertyDescriptor SNMP_CLIENT_PORT = new PropertyDescriptor.Builder()
+            .name("snmp-client-port")
+            .displayName("SNMP client (processor) port")

Review comment:
       Recommend removing the `(processor)` portion and updating the description if needed.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/pom.xml
##########
@@ -30,13 +31,18 @@ language governing permissions and limitations under the License. -->
             <artifactId>snmp4j</artifactId>
             <version>${snmp4j.version}</version>
         </dependency>
-        
         <dependency>
             <groupId>org.snmp4j</groupId>
             <artifactId>snmp4j-agent</artifactId>
             <version>${snmp4j-agent.version}</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.snmp4j</groupId>
+            <artifactId>snmp4j-log4j</artifactId>

Review comment:
       Is there a particular reason for introducing `snmp4j-log4j`?  At minimum, the `log4j-core` transitive dependency should be excluded and replaced with the appropriate SLF4J wrapper so that all logging calls go through SLF4J.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/logging/Slf4jLogAdapter.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.nifi.snmp.logging;
+
+import org.slf4j.Logger;
+import org.snmp4j.log.LogAdapter;
+import org.snmp4j.log.LogLevel;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.logging.Handler;
+
+public class Slf4jLogAdapter implements LogAdapter {
+
+    private final Logger logger;
+
+    public Slf4jLogAdapter(Logger logger) {
+        this.logger = logger;
+    }
+
+    // ---- Checking methods
+
+    public boolean isDebugEnabled() {
+        return true;

Review comment:
       Should this and other status level checks call the associated SLF4J Logger method?
   ```suggestion
           return logger.isDebugEnabled();
   ```

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/operations/SNMPSetter.java
##########
@@ -26,31 +26,30 @@
 import org.snmp4j.event.ResponseEvent;
 
 /**
- * Extension of {@link SNMPWorker} to perform SNMP Set requests
+ * Extension of {@link SNMPWorker} to perform SNMP Set requests.
  */
-final class SNMPSetter extends SNMPWorker {
+public final class SNMPSetter extends SNMPWorker {
 
-    /** logger */
-    private final static Logger logger = LoggerFactory.getLogger(SNMPSetter.class);
+    private static final Logger LOGGER = LoggerFactory.getLogger(SNMPSetter.class);
 
     /**
-     * Creates an instance of this setter
+     * Creates an instance of this setter.
      * @param snmp instance of {@link Snmp}
      * @param target instance of {@link AbstractTarget} to request
      */
-    SNMPSetter(Snmp snmp, AbstractTarget target) {
+    public SNMPSetter(Snmp snmp, AbstractTarget target) {
         super(snmp, target);
-        logger.info("Successfully initialized SNMP Setter");
+        LOGGER.info("Successfully initialized SNMP Setter");

Review comment:
       As noted on `SNMPGetter`, this seems more appropriate as a debug log.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/logging/Slf4jLogAdapter.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.nifi.snmp.logging;
+
+import org.slf4j.Logger;
+import org.snmp4j.log.LogAdapter;
+import org.snmp4j.log.LogLevel;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.logging.Handler;
+
+public class Slf4jLogAdapter implements LogAdapter {
+
+    private final Logger logger;
+
+    public Slf4jLogAdapter(Logger logger) {
+        this.logger = logger;
+    }
+
+    // ---- Checking methods
+
+    public boolean isDebugEnabled() {
+        return true;
+    }
+
+    public boolean isInfoEnabled() {
+        return true;
+    }
+
+    public boolean isWarnEnabled() {
+        return true;
+    }
+
+    // ---- Logging methods
+
+    public void debug(Serializable message) {
+        log(LogLevel.DEBUG, message.toString(), null);
+    }
+
+    public void info(CharSequence message) {
+        log(LogLevel.INFO, message.toString(), null);
+    }
+
+    public void warn(Serializable message) {
+        log(LogLevel.WARN, message.toString(), null);
+    }
+
+    public void error(Serializable message) {
+        log(LogLevel.ERROR, message.toString(), null);
+    }
+
+    public void error(CharSequence message, Throwable t) {
+        log(LogLevel.ERROR, message.toString(), t);
+    }
+
+    public void fatal(Object message) {
+        log(LogLevel.FATAL, message.toString(), null);
+    }
+
+    public void fatal(CharSequence message, Throwable t) {
+        log(LogLevel.FATAL, message.toString(), t);
+    }
+
+    // ---- Public methods
+
+    public LogLevel getEffectiveLogLevel() {
+        return LogLevel.ALL;

Review comment:
       Is there a reason for returning `ALL` as opposed to interrogating the SLF4J Logger and returning the appropriate value?

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/operations/SNMPGetter.java
##########
@@ -35,63 +35,62 @@
 import org.snmp4j.util.TreeUtils;
 
 /**
- * Extension of {@link SNMPWorker} to perform SNMP Get and SNMP Walk requests
+ * Extension of {@link SNMPWorker} to perform SNMP Get and SNMP Walk requests.
  */
-final class SNMPGetter extends SNMPWorker {
+public final class SNMPGetter extends SNMPWorker {
 
-    /** logger */
-    private final static Logger logger = LoggerFactory.getLogger(SNMPGetter.class);
-
-    /** OID to request */
+    private static final Logger LOGGER = LoggerFactory.getLogger(SNMPGetter.class);
     private final OID oid;
 
     /**
-     * Creates an instance of this getter
-     * @param snmp instance of {@link Snmp}
+     * Creates an instance of this getter.
+     *
+     * @param snmp   instance of {@link Snmp}
      * @param target instance of {@link AbstractTarget} to request
-     * @param oid instance of {@link OID} to request
+     * @param oid    instance of {@link OID} to request
      */
-    SNMPGetter(Snmp snmp, AbstractTarget target, OID oid) {
+    public SNMPGetter(Snmp snmp, AbstractTarget target, OID oid) {
         super(snmp, target);
         this.oid = oid;
-        logger.info("Successfully initialized SNMP Getter");
+        LOGGER.info("Successfully initialized SNMP Getter");

Review comment:
       Although this was already an info message, it appears that it would be more appropriate to log at the debug level, and include some details from the parameters provided.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/logging/Slf4jLogAdapter.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.nifi.snmp.logging;
+
+import org.slf4j.Logger;
+import org.snmp4j.log.LogAdapter;
+import org.snmp4j.log.LogLevel;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.logging.Handler;
+
+public class Slf4jLogAdapter implements LogAdapter {
+
+    private final Logger logger;
+
+    public Slf4jLogAdapter(Logger logger) {
+        this.logger = logger;
+    }
+
+    // ---- Checking methods
+
+    public boolean isDebugEnabled() {
+        return true;
+    }
+
+    public boolean isInfoEnabled() {
+        return true;
+    }
+
+    public boolean isWarnEnabled() {
+        return true;
+    }
+
+    // ---- Logging methods
+
+    public void debug(Serializable message) {
+        log(LogLevel.DEBUG, message.toString(), null);
+    }
+
+    public void info(CharSequence message) {
+        log(LogLevel.INFO, message.toString(), null);
+    }
+
+    public void warn(Serializable message) {
+        log(LogLevel.WARN, message.toString(), null);
+    }
+
+    public void error(Serializable message) {
+        log(LogLevel.ERROR, message.toString(), null);
+    }
+
+    public void error(CharSequence message, Throwable t) {
+        log(LogLevel.ERROR, message.toString(), t);
+    }
+
+    public void fatal(Object message) {
+        log(LogLevel.FATAL, message.toString(), null);
+    }
+
+    public void fatal(CharSequence message, Throwable t) {
+        log(LogLevel.FATAL, message.toString(), t);
+    }
+
+    // ---- Public methods
+
+    public LogLevel getEffectiveLogLevel() {
+        return LogLevel.ALL;
+    }
+
+    public Iterator<Handler> getLogHandler() {
+        return null;
+    }
+
+    public LogLevel getLogLevel() {
+        return getEffectiveLogLevel();
+    }
+
+    public String getName() {
+        return logger.getName();
+    }
+
+    public void setLogLevel(LogLevel logLevel) {
+        // no need to set log level
+    }
+
+    // ---- Private methods
+
+    private void log(LogLevel logLevel, String msg, Throwable t) {

Review comment:
       Is there a particular reason for having this private method with the `LogLevel` parameter as opposed to just calling the appropriate SLF4J Logger method in the associated LogAdapter method?

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/processors/SetSNMP.java
##########
@@ -16,209 +16,127 @@
  */
 package org.apache.nifi.snmp.processors;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.snmp.operations.SNMPSetter;
+import org.apache.nifi.snmp.utils.SNMPUtils;
 import org.snmp4j.PDU;
 import org.snmp4j.ScopedPDU;
 import org.snmp4j.event.ResponseEvent;
 import org.snmp4j.mp.SnmpConstants;
-import org.snmp4j.smi.AbstractVariable;
-import org.snmp4j.smi.AssignableFromInteger;
-import org.snmp4j.smi.AssignableFromLong;
-import org.snmp4j.smi.AssignableFromString;
-import org.snmp4j.smi.OID;
-import org.snmp4j.smi.OctetString;
-import org.snmp4j.smi.Variable;
-import org.snmp4j.smi.VariableBinding;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 /**
  * Performs a SNMP Set operation based on attributes of incoming FlowFile.
  * Upon each invocation of {@link #onTrigger(ProcessContext, ProcessSession)}
  * method, it will inspect attributes of FlowFile and look for attributes with
  * name formatted as "snmp$OID" to set the attribute value to this OID.
  */
-@Tags({ "snmp", "set", "oid" })
+@Tags({"snmp", "set", "oid"})
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @CapabilityDescription("Based on incoming FlowFile attributes, the processor will execute SNMP Set requests." +
         " When founding attributes with name like snmp$<OID>, the processor will atempt to set the value of" +
         " attribute to the corresponding OID given in the attribute name")
-public class SetSNMP extends AbstractSNMPProcessor<SNMPSetter> {
+public class SetSNMP extends AbstractSNMPProcessor {
 
-    /** relationship for success */
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
             .description("All FlowFiles that have been successfully used to perform SNMP Set are routed to this relationship")
             .build();
-    /** relationship for failure */
+
     public static final Relationship REL_FAILURE = new Relationship.Builder()
             .name("failure")
             .description("All FlowFiles that failed during the SNMP Set care routed to this relationship")
             .build();
 
-    /** list of properties descriptors */
-    private final static List<PropertyDescriptor> propertyDescriptors;
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
 
-    /** list of relationships */
-    private final static Set<Relationship> relationships;
-
-    /*
-     * Will ensure that the list of property descriptors is build only once.
-     * Will also create a Set of relationships
-     */
-    static {
-        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
-        _propertyDescriptors.addAll(descriptors);
-        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+    private SNMPSetter snmpSetter;
 
-        Set<Relationship> _relationships = new HashSet<>();
-        _relationships.add(REL_SUCCESS);
-        _relationships.add(REL_FAILURE);
-        relationships = Collections.unmodifiableSet(_relationships);
+    @OnScheduled
+    public void initSnmpClient(ProcessContext context) {
+        super.initSnmpClient(context);
+        snmpSetter = new SNMPSetter(snmpContext.getSnmp(), snmpContext.getTarget());
     }
 
-    /**
-     * @see org.apache.nifi.snmp.processors.AbstractSNMPProcessor#onTriggerSnmp(org.apache.nifi.processor.ProcessContext, org.apache.nifi.processor.ProcessSession)
-     */
+
     @Override
-    protected void onTriggerSnmp(ProcessContext context, ProcessSession processSession) throws ProcessException {
+    public void onTrigger(ProcessContext context, ProcessSession processSession) {
         FlowFile flowFile = processSession.get();
         if (flowFile != null) {
-            // Create the PDU object
-            PDU pdu = null;
-            if(this.snmpTarget.getVersion() == SnmpConstants.version3) {
-                pdu = new ScopedPDU();
-            } else {
-                pdu = new PDU();
-            }
-            if(this.addVariables(pdu, flowFile.getAttributes())) {
+            PDU pdu = createPdu();
+            if (SNMPUtils.addVariables(pdu, flowFile.getAttributes(), getLogger())) {
                 pdu.setType(PDU.SET);
-                try {
-                    ResponseEvent response = this.targetResource.set(pdu);
-                    if(response.getResponse() == null) {
-                        processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                        this.getLogger().error("Set request timed out or parameters are incorrect.");
-                        context.yield();
-                    } else if(response.getResponse().getErrorStatus() == PDU.noError) {
-                        flowFile = SNMPUtils.updateFlowFileAttributesWithPduProperties(pdu, flowFile, processSession);
-                        processSession.transfer(flowFile, REL_SUCCESS);
-                        processSession.getProvenanceReporter().send(flowFile, this.snmpTarget.getAddress().toString());
-                    } else {
-                        final String error = response.getResponse().getErrorStatusText();
-                        flowFile = SNMPUtils.addAttribute(SNMPUtils.SNMP_PROP_PREFIX + "error", error, flowFile, processSession);
-                        processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                        this.getLogger().error("Failed while executing SNMP Set [{}] via " + this.targetResource + ". Error = {}", new Object[]{response.getRequest().getVariableBindings(), error});
-                    }
-                } catch (IOException e) {
-                    processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                    this.getLogger().error("Failed while executing SNMP Set via " + this.targetResource, e);
-                    context.yield();
-                }
+                processPdu(context, processSession, flowFile, pdu);
             } else {
                 processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                this.getLogger().warn("No attributes found in the FlowFile to perform SNMP Set");
+                getLogger().warn("No attributes found in the FlowFile to perform SNMP Set");
             }
         }
     }
 
-    /**
-     * Method to construct {@link VariableBinding} based on {@link FlowFile}
-     * attributes in order to update the {@link PDU} that is going to be sent to
-     * the SNMP Agent.
-     * @param pdu {@link PDU} to be sent
-     * @param attributes {@link FlowFile} attributes
-     * @return true if at least one {@link VariableBinding} has been created, false otherwise
-     */
-    private boolean addVariables(PDU pdu, Map<String, String> attributes) {
-        boolean result = false;
-        for (Entry<String, String> attributeEntry : attributes.entrySet()) {
-            if (attributeEntry.getKey().startsWith(SNMPUtils.SNMP_PROP_PREFIX)) {
-                String[] splits = attributeEntry.getKey().split("\\" + SNMPUtils.SNMP_PROP_DELIMITER);
-                String snmpPropName = splits[1];
-                String snmpPropValue = attributeEntry.getValue();
-                if(SNMPUtils.OID_PATTERN.matcher(snmpPropName).matches()) {
-                    Variable var = null;
-                    if (splits.length == 2) { // no SMI syntax defined
-                        var = new OctetString(snmpPropValue);
-                    } else {
-                        int smiSyntax = Integer.valueOf(splits[2]);
-                        var = this.stringToVariable(snmpPropValue, smiSyntax);
-                    }
-                    if(var != null) {
-                        VariableBinding varBind = new VariableBinding(new OID(snmpPropName), var);
-                        pdu.add(varBind);
-                        result = true;
-                    }
-                }
+    private void processPdu(ProcessContext context, ProcessSession processSession, FlowFile flowFile, PDU pdu) {
+        try {
+            ResponseEvent response = snmpSetter.set(pdu);
+            if (response.getResponse() == null) {
+                processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
+                getLogger().error("Set request timed out or parameters are incorrect.");

Review comment:
       Logging FlowFile or some identifying attributes would be helpful for troubleshooting.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/processors/AbstractSNMPProcessor.java
##########
@@ -16,66 +16,69 @@
  */
 package org.apache.nifi.snmp.processors;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.snmp4j.AbstractTarget;
-import org.snmp4j.CommunityTarget;
-import org.snmp4j.Snmp;
-import org.snmp4j.TransportMapping;
-import org.snmp4j.UserTarget;
-import org.snmp4j.mp.MPv3;
-import org.snmp4j.mp.SnmpConstants;
-import org.snmp4j.security.SecurityModels;
-import org.snmp4j.security.SecurityProtocols;
-import org.snmp4j.security.USM;
-import org.snmp4j.security.UsmUser;
-import org.snmp4j.smi.OctetString;
-import org.snmp4j.smi.UdpAddress;
-import org.snmp4j.transport.DefaultUdpTransportMapping;
+import org.apache.nifi.snmp.configuration.BasicConfiguration;
+import org.apache.nifi.snmp.configuration.SecurityConfiguration;
+import org.apache.nifi.snmp.configuration.SecurityConfigurationBuilder;
+import org.apache.nifi.snmp.context.SNMPContext;
+import org.apache.nifi.snmp.logging.Slf4jLogFactory;
+import org.apache.nifi.snmp.validators.OIDValidator;
+import org.snmp4j.log.LogFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
 
 /**
- * Base processor that uses SNMP4J client API
+ * Base processor that uses SNMP4J client API.
  * (http://www.snmp4j.org/)
- *
- * @param <T> the type of {@link SNMPWorker}. Please see {@link SNMPSetter}
- *            and {@link SNMPGetter}
  */
-abstract class AbstractSNMPProcessor<T extends SNMPWorker> extends AbstractProcessor {
+abstract class AbstractSNMPProcessor extends AbstractProcessor {
+
+    static {
+        LogFactory.setLogFactory(new Slf4jLogFactory());
+    }
+
+    // Property to define the host of the SNMP agent.
+    public static final PropertyDescriptor SNMP_CLIENT_PORT = new PropertyDescriptor.Builder()
+            .name("snmp-client-port")
+            .displayName("SNMP client (processor) port")
+            .description("The processor runs an SNMP client on localhost. The port however can be specified")
+            .required(true)
+            .defaultValue("0")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)

Review comment:
       It looks like the `PORT_VALIDATOR` would provide better validation to avoid invalid port numbers.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/processors/SetSNMP.java
##########
@@ -16,209 +16,127 @@
  */
 package org.apache.nifi.snmp.processors;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.snmp.operations.SNMPSetter;
+import org.apache.nifi.snmp.utils.SNMPUtils;
 import org.snmp4j.PDU;
 import org.snmp4j.ScopedPDU;
 import org.snmp4j.event.ResponseEvent;
 import org.snmp4j.mp.SnmpConstants;
-import org.snmp4j.smi.AbstractVariable;
-import org.snmp4j.smi.AssignableFromInteger;
-import org.snmp4j.smi.AssignableFromLong;
-import org.snmp4j.smi.AssignableFromString;
-import org.snmp4j.smi.OID;
-import org.snmp4j.smi.OctetString;
-import org.snmp4j.smi.Variable;
-import org.snmp4j.smi.VariableBinding;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 /**
  * Performs a SNMP Set operation based on attributes of incoming FlowFile.
  * Upon each invocation of {@link #onTrigger(ProcessContext, ProcessSession)}
  * method, it will inspect attributes of FlowFile and look for attributes with
  * name formatted as "snmp$OID" to set the attribute value to this OID.
  */
-@Tags({ "snmp", "set", "oid" })
+@Tags({"snmp", "set", "oid"})
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @CapabilityDescription("Based on incoming FlowFile attributes, the processor will execute SNMP Set requests." +
         " When founding attributes with name like snmp$<OID>, the processor will atempt to set the value of" +
         " attribute to the corresponding OID given in the attribute name")
-public class SetSNMP extends AbstractSNMPProcessor<SNMPSetter> {
+public class SetSNMP extends AbstractSNMPProcessor {
 
-    /** relationship for success */
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
             .description("All FlowFiles that have been successfully used to perform SNMP Set are routed to this relationship")
             .build();
-    /** relationship for failure */
+
     public static final Relationship REL_FAILURE = new Relationship.Builder()
             .name("failure")
             .description("All FlowFiles that failed during the SNMP Set care routed to this relationship")
             .build();
 
-    /** list of properties descriptors */
-    private final static List<PropertyDescriptor> propertyDescriptors;
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
 
-    /** list of relationships */
-    private final static Set<Relationship> relationships;
-
-    /*
-     * Will ensure that the list of property descriptors is build only once.
-     * Will also create a Set of relationships
-     */
-    static {
-        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
-        _propertyDescriptors.addAll(descriptors);
-        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+    private SNMPSetter snmpSetter;
 
-        Set<Relationship> _relationships = new HashSet<>();
-        _relationships.add(REL_SUCCESS);
-        _relationships.add(REL_FAILURE);
-        relationships = Collections.unmodifiableSet(_relationships);
+    @OnScheduled
+    public void initSnmpClient(ProcessContext context) {
+        super.initSnmpClient(context);
+        snmpSetter = new SNMPSetter(snmpContext.getSnmp(), snmpContext.getTarget());
     }
 
-    /**
-     * @see org.apache.nifi.snmp.processors.AbstractSNMPProcessor#onTriggerSnmp(org.apache.nifi.processor.ProcessContext, org.apache.nifi.processor.ProcessSession)
-     */
+
     @Override
-    protected void onTriggerSnmp(ProcessContext context, ProcessSession processSession) throws ProcessException {
+    public void onTrigger(ProcessContext context, ProcessSession processSession) {
         FlowFile flowFile = processSession.get();
         if (flowFile != null) {
-            // Create the PDU object
-            PDU pdu = null;
-            if(this.snmpTarget.getVersion() == SnmpConstants.version3) {
-                pdu = new ScopedPDU();
-            } else {
-                pdu = new PDU();
-            }
-            if(this.addVariables(pdu, flowFile.getAttributes())) {
+            PDU pdu = createPdu();
+            if (SNMPUtils.addVariables(pdu, flowFile.getAttributes(), getLogger())) {
                 pdu.setType(PDU.SET);
-                try {
-                    ResponseEvent response = this.targetResource.set(pdu);
-                    if(response.getResponse() == null) {
-                        processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                        this.getLogger().error("Set request timed out or parameters are incorrect.");
-                        context.yield();
-                    } else if(response.getResponse().getErrorStatus() == PDU.noError) {
-                        flowFile = SNMPUtils.updateFlowFileAttributesWithPduProperties(pdu, flowFile, processSession);
-                        processSession.transfer(flowFile, REL_SUCCESS);
-                        processSession.getProvenanceReporter().send(flowFile, this.snmpTarget.getAddress().toString());
-                    } else {
-                        final String error = response.getResponse().getErrorStatusText();
-                        flowFile = SNMPUtils.addAttribute(SNMPUtils.SNMP_PROP_PREFIX + "error", error, flowFile, processSession);
-                        processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                        this.getLogger().error("Failed while executing SNMP Set [{}] via " + this.targetResource + ". Error = {}", new Object[]{response.getRequest().getVariableBindings(), error});
-                    }
-                } catch (IOException e) {
-                    processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                    this.getLogger().error("Failed while executing SNMP Set via " + this.targetResource, e);
-                    context.yield();
-                }
+                processPdu(context, processSession, flowFile, pdu);
             } else {
                 processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                this.getLogger().warn("No attributes found in the FlowFile to perform SNMP Set");
+                getLogger().warn("No attributes found in the FlowFile to perform SNMP Set");

Review comment:
       It would be helpful to log the FlowFile object or identifier for troubleshooting.
   ```suggestion
                   getLogger().warn("No attributes found in {} to perform SNMP Set", flowFile);
   ```

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/processors/GetSNMP.java
##########
@@ -87,113 +91,110 @@
             .defaultValue("GET")
             .build();
 
-    /** relationship for success */
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
             .description("All FlowFiles that are received from the SNMP agent are routed to this relationship")
             .build();
 
-    /** relationship for failure */
     public static final Relationship REL_FAILURE = new Relationship.Builder()
             .name("failure")
             .description("All FlowFiles that cannot received from the SNMP agent are routed to this relationship")
             .build();
 
-    /** list of property descriptors */
-    private final static List<PropertyDescriptor> propertyDescriptors;
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyList();
 
-    /** list of relationships */
-    private final static Set<Relationship> relationships;
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
 
-    /*
-     * Will ensure that the list of property descriptors is build only once.
-     * Will also create a Set of relationships
-     */
-    static {
-        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
-        _propertyDescriptors.add(OID);
-        _propertyDescriptors.add(TEXTUAL_OID);
-        _propertyDescriptors.add(SNMP_STRATEGY);
-        _propertyDescriptors.addAll(descriptors);
-        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
-
-        Set<Relationship> _relationships = new HashSet<>();
-        _relationships.add(REL_SUCCESS);
-        _relationships.add(REL_FAILURE);
-        relationships = Collections.unmodifiableSet(_relationships);
+    private SNMPGetter snmpGetter;
+
+    @OnScheduled
+    @Override
+    public void initSnmpClient(ProcessContext context) {
+        super.initSnmpClient(context);
+        String oid = context.getProperty(OID).getValue();
+        snmpGetter = new SNMPGetter(snmpContext.getSnmp(), snmpContext.getTarget(), new OID(oid));
     }
 
     /**
      * Delegate method to supplement
      * {@link #onTrigger(ProcessContext, ProcessSession)}. It is implemented by
      * sub-classes to perform {@link Processor} specific functionality.
      *
-     * @param context
-     *            instance of {@link ProcessContext}
-     * @param processSession
-     *            instance of {@link ProcessSession}
+     * @param context        instance of {@link ProcessContext}
+     * @param processSession instance of {@link ProcessSession}
      * @throws ProcessException Process exception
      */
     @Override
-    protected void onTriggerSnmp(ProcessContext context, ProcessSession processSession) throws ProcessException {
-        if("GET".equals(context.getProperty(SNMP_STRATEGY).getValue())) {
-            final ResponseEvent response = this.targetResource.get();
-            if (response.getResponse() != null){
-                FlowFile flowFile = processSession.create();
+    public void onTrigger(ProcessContext context, ProcessSession processSession) {
+        final String targetUri = snmpContext.getTarget().getAddress().toString();
+        final String snmpStrategy = context.getProperty(SNMP_STRATEGY).getValue();
+        final String oid = context.getProperty(OID).getValue();
+
+        if ("GET".equals(snmpStrategy)) {

Review comment:
       With the `SNMP_STRATEGY` taking a list of allowable values, it would be helpful to define `GET` and `WALK` as either static variables or perhaps create an enum for reuse in both the property definition and in this method.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/validators/OIDValidator.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.nifi.snmp.validators;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.snmp.configuration.SecurityConfiguration;
+import org.apache.nifi.util.StringUtils;
+import org.snmp4j.security.SecurityLevel;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+
+public class OIDValidator {
+
+    private static final String SNMP_V3 = "SNMPv3";

Review comment:
       The SNMP version strings appear to be used in several places so it could be useful to define a reusable enum.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/helper/SNMPTestUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.nifi.snmp.helper;
+
+import org.apache.nifi.flowfile.FlowFile;
+
+import java.net.ServerSocket;
+import java.util.HashMap;
+import java.util.Map;
+
+public class SNMPTestUtil {
+
+    /**
+     * Will determine an available port.
+     */
+    public static synchronized int availablePort() {
+        try (ServerSocket s = new ServerSocket(0)) {
+            s.setReuseAddress(true);
+            return s.getLocalPort();
+        } catch (Exception e) {
+            throw new IllegalStateException("Failed to discover available port.", e);
+        }
+    }
+
+    public static FlowFile createFlowFile(final long id, final long fileSize, final Map<String, String> attributes) {

Review comment:
       Is there a reason for using this method as opposed to `MockFlowFile`?

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/helper/SNMPTestUtil.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.nifi.snmp.helper;
+
+import org.apache.nifi.flowfile.FlowFile;
+
+import java.net.ServerSocket;
+import java.util.HashMap;
+import java.util.Map;
+
+public class SNMPTestUtil {
+
+    /**
+     * Will determine an available port.
+     */
+    public static synchronized int availablePort() {

Review comment:
       The `NetworkUtils` class in `nifi-utils` contains this same method, is there a reason for redefining it here?

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/testagents/TestSNMPV1Agent.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.nifi.snmp.testagents;
+
+import org.apache.nifi.snmp.helper.SNMPTestUtil;
+import org.snmp4j.TransportMapping;
+import org.snmp4j.agent.BaseAgent;
+import org.snmp4j.agent.CommandProcessor;
+import org.snmp4j.agent.DuplicateRegistrationException;
+import org.snmp4j.agent.MOGroup;
+import org.snmp4j.agent.ManagedObject;
+import org.snmp4j.agent.mo.snmp.RowStatus;
+import org.snmp4j.agent.mo.snmp.SnmpCommunityMIB;
+import org.snmp4j.agent.mo.snmp.SnmpNotificationMIB;
+import org.snmp4j.agent.mo.snmp.SnmpTargetMIB;
+import org.snmp4j.agent.mo.snmp.StorageType;
+import org.snmp4j.agent.mo.snmp.VacmMIB;
+import org.snmp4j.agent.security.MutableVACM;
+import org.snmp4j.log.ConsoleLogFactory;
+import org.snmp4j.log.LogFactory;
+import org.snmp4j.mp.MPv3;
+import org.snmp4j.security.SecurityLevel;
+import org.snmp4j.security.SecurityModel;
+import org.snmp4j.security.USM;
+import org.snmp4j.smi.Address;
+import org.snmp4j.smi.GenericAddress;
+import org.snmp4j.smi.Integer32;
+import org.snmp4j.smi.OID;
+import org.snmp4j.smi.OctetString;
+import org.snmp4j.smi.Variable;
+import org.snmp4j.transport.TransportMappings;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+
+public class TestSNMPV1Agent extends BaseAgent {
+
+    static {
+        LogFactory.setLogFactory(new ConsoleLogFactory());
+        //ConsoleLogAdapter.setDebugEnabled(true);

Review comment:
       Is there a reason for keeping this comment, or should it be removed?

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/processors/GetSNMP.java
##########
@@ -87,113 +91,110 @@
             .defaultValue("GET")
             .build();
 
-    /** relationship for success */
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
             .description("All FlowFiles that are received from the SNMP agent are routed to this relationship")
             .build();
 
-    /** relationship for failure */
     public static final Relationship REL_FAILURE = new Relationship.Builder()
             .name("failure")
             .description("All FlowFiles that cannot received from the SNMP agent are routed to this relationship")
             .build();
 
-    /** list of property descriptors */
-    private final static List<PropertyDescriptor> propertyDescriptors;
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyList();
 
-    /** list of relationships */
-    private final static Set<Relationship> relationships;
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
 
-    /*
-     * Will ensure that the list of property descriptors is build only once.
-     * Will also create a Set of relationships
-     */
-    static {
-        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
-        _propertyDescriptors.add(OID);
-        _propertyDescriptors.add(TEXTUAL_OID);
-        _propertyDescriptors.add(SNMP_STRATEGY);
-        _propertyDescriptors.addAll(descriptors);
-        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
-
-        Set<Relationship> _relationships = new HashSet<>();
-        _relationships.add(REL_SUCCESS);
-        _relationships.add(REL_FAILURE);
-        relationships = Collections.unmodifiableSet(_relationships);
+    private SNMPGetter snmpGetter;
+
+    @OnScheduled
+    @Override
+    public void initSnmpClient(ProcessContext context) {
+        super.initSnmpClient(context);
+        String oid = context.getProperty(OID).getValue();
+        snmpGetter = new SNMPGetter(snmpContext.getSnmp(), snmpContext.getTarget(), new OID(oid));
     }
 
     /**
      * Delegate method to supplement
      * {@link #onTrigger(ProcessContext, ProcessSession)}. It is implemented by
      * sub-classes to perform {@link Processor} specific functionality.
      *
-     * @param context
-     *            instance of {@link ProcessContext}
-     * @param processSession
-     *            instance of {@link ProcessSession}
+     * @param context        instance of {@link ProcessContext}
+     * @param processSession instance of {@link ProcessSession}
      * @throws ProcessException Process exception
      */
     @Override
-    protected void onTriggerSnmp(ProcessContext context, ProcessSession processSession) throws ProcessException {
-        if("GET".equals(context.getProperty(SNMP_STRATEGY).getValue())) {
-            final ResponseEvent response = this.targetResource.get();
-            if (response.getResponse() != null){
-                FlowFile flowFile = processSession.create();
+    public void onTrigger(ProcessContext context, ProcessSession processSession) {
+        final String targetUri = snmpContext.getTarget().getAddress().toString();
+        final String snmpStrategy = context.getProperty(SNMP_STRATEGY).getValue();
+        final String oid = context.getProperty(OID).getValue();
+
+        if ("GET".equals(snmpStrategy)) {
+            final ResponseEvent response = snmpGetter.get();
+            if (response.getResponse() != null) {
                 PDU pdu = response.getResponse();
-                flowFile = SNMPUtils.updateFlowFileAttributesWithPduProperties(pdu, flowFile, processSession);
-                flowFile = SNMPUtils.addAttribute(SNMPUtils.SNMP_PROP_PREFIX + "textualOid",
-                        context.getProperty(TEXTUAL_OID).getValue(), flowFile, processSession);
-                processSession.getProvenanceReporter().receive(flowFile,
-                        this.snmpTarget.getAddress().toString() + "/" + context.getProperty(OID).getValue());
-                if(pdu.getErrorStatus() == PDU.noError) {
+                FlowFile flowFile = createFlowFile(context, processSession, pdu);
+                processSession.getProvenanceReporter().receive(flowFile, targetUri + "/" + oid);
+                if (pdu.getErrorStatus() == PDU.noError) {
                     processSession.transfer(flowFile, REL_SUCCESS);
                 } else {
                     processSession.transfer(flowFile, REL_FAILURE);
                 }
             } else {
-                this.getLogger().error("Get request timed out or parameters are incorrect.");
+                getLogger().error("Get request timed out or parameters are incorrect.");
                 context.yield();
             }
-        } else if("WALK".equals(context.getProperty(SNMP_STRATEGY).getValue())) {
-            final List<TreeEvent> events = this.targetResource.walk();
-            if((events != null) && !events.isEmpty() && (events.get(0).getVariableBindings() != null)) {
+        } else if ("WALK".equals(snmpStrategy)) {
+            final List<TreeEvent> events = snmpGetter.walk();
+            if (areValidEvents(events)) {
                 FlowFile flowFile = processSession.create();
                 for (TreeEvent treeEvent : events) {
                     flowFile = SNMPUtils.updateFlowFileAttributesWithTreeEventProperties(treeEvent, flowFile, processSession);
                 }
-                processSession.getProvenanceReporter().receive(flowFile,
-                        this.snmpTarget.getAddress().toString() + "/" + context.getProperty(OID).getValue());
+                processSession.getProvenanceReporter().receive(flowFile, targetUri + "/" + oid);
                 processSession.transfer(flowFile, REL_SUCCESS);
             } else {
-                this.getLogger().error("Get request timed out or parameters are incorrect.");
+                getLogger().error("Get request timed out or parameters are incorrect.");

Review comment:
       Both this error log and the error log for `GET` contain the same message.  It would be helpful to differentiate these two log messages with some additional parameters, including at least the strategy used.
   ```suggestion
                   getLogger().error("SNMP WALK Failed: Request timed out or parameters may be invalid");
   ```

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/processors/SetSNMP.java
##########
@@ -16,209 +16,127 @@
  */
 package org.apache.nifi.snmp.processors;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.snmp.operations.SNMPSetter;
+import org.apache.nifi.snmp.utils.SNMPUtils;
 import org.snmp4j.PDU;
 import org.snmp4j.ScopedPDU;
 import org.snmp4j.event.ResponseEvent;
 import org.snmp4j.mp.SnmpConstants;
-import org.snmp4j.smi.AbstractVariable;
-import org.snmp4j.smi.AssignableFromInteger;
-import org.snmp4j.smi.AssignableFromLong;
-import org.snmp4j.smi.AssignableFromString;
-import org.snmp4j.smi.OID;
-import org.snmp4j.smi.OctetString;
-import org.snmp4j.smi.Variable;
-import org.snmp4j.smi.VariableBinding;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 /**
  * Performs a SNMP Set operation based on attributes of incoming FlowFile.
  * Upon each invocation of {@link #onTrigger(ProcessContext, ProcessSession)}
  * method, it will inspect attributes of FlowFile and look for attributes with
  * name formatted as "snmp$OID" to set the attribute value to this OID.
  */
-@Tags({ "snmp", "set", "oid" })
+@Tags({"snmp", "set", "oid"})
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @CapabilityDescription("Based on incoming FlowFile attributes, the processor will execute SNMP Set requests." +
         " When founding attributes with name like snmp$<OID>, the processor will atempt to set the value of" +
         " attribute to the corresponding OID given in the attribute name")
-public class SetSNMP extends AbstractSNMPProcessor<SNMPSetter> {
+public class SetSNMP extends AbstractSNMPProcessor {
 
-    /** relationship for success */
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
             .description("All FlowFiles that have been successfully used to perform SNMP Set are routed to this relationship")
             .build();
-    /** relationship for failure */
+
     public static final Relationship REL_FAILURE = new Relationship.Builder()
             .name("failure")
             .description("All FlowFiles that failed during the SNMP Set care routed to this relationship")
             .build();
 
-    /** list of properties descriptors */
-    private final static List<PropertyDescriptor> propertyDescriptors;
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
 
-    /** list of relationships */
-    private final static Set<Relationship> relationships;
-
-    /*
-     * Will ensure that the list of property descriptors is build only once.
-     * Will also create a Set of relationships
-     */
-    static {
-        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
-        _propertyDescriptors.addAll(descriptors);
-        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+    private SNMPSetter snmpSetter;
 
-        Set<Relationship> _relationships = new HashSet<>();
-        _relationships.add(REL_SUCCESS);
-        _relationships.add(REL_FAILURE);
-        relationships = Collections.unmodifiableSet(_relationships);
+    @OnScheduled
+    public void initSnmpClient(ProcessContext context) {
+        super.initSnmpClient(context);
+        snmpSetter = new SNMPSetter(snmpContext.getSnmp(), snmpContext.getTarget());
     }
 
-    /**
-     * @see org.apache.nifi.snmp.processors.AbstractSNMPProcessor#onTriggerSnmp(org.apache.nifi.processor.ProcessContext, org.apache.nifi.processor.ProcessSession)
-     */
+
     @Override
-    protected void onTriggerSnmp(ProcessContext context, ProcessSession processSession) throws ProcessException {
+    public void onTrigger(ProcessContext context, ProcessSession processSession) {
         FlowFile flowFile = processSession.get();
         if (flowFile != null) {
-            // Create the PDU object
-            PDU pdu = null;
-            if(this.snmpTarget.getVersion() == SnmpConstants.version3) {
-                pdu = new ScopedPDU();
-            } else {
-                pdu = new PDU();
-            }
-            if(this.addVariables(pdu, flowFile.getAttributes())) {
+            PDU pdu = createPdu();
+            if (SNMPUtils.addVariables(pdu, flowFile.getAttributes(), getLogger())) {
                 pdu.setType(PDU.SET);
-                try {
-                    ResponseEvent response = this.targetResource.set(pdu);
-                    if(response.getResponse() == null) {
-                        processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                        this.getLogger().error("Set request timed out or parameters are incorrect.");
-                        context.yield();
-                    } else if(response.getResponse().getErrorStatus() == PDU.noError) {
-                        flowFile = SNMPUtils.updateFlowFileAttributesWithPduProperties(pdu, flowFile, processSession);
-                        processSession.transfer(flowFile, REL_SUCCESS);
-                        processSession.getProvenanceReporter().send(flowFile, this.snmpTarget.getAddress().toString());
-                    } else {
-                        final String error = response.getResponse().getErrorStatusText();
-                        flowFile = SNMPUtils.addAttribute(SNMPUtils.SNMP_PROP_PREFIX + "error", error, flowFile, processSession);
-                        processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                        this.getLogger().error("Failed while executing SNMP Set [{}] via " + this.targetResource + ". Error = {}", new Object[]{response.getRequest().getVariableBindings(), error});
-                    }
-                } catch (IOException e) {
-                    processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                    this.getLogger().error("Failed while executing SNMP Set via " + this.targetResource, e);
-                    context.yield();
-                }
+                processPdu(context, processSession, flowFile, pdu);
             } else {
                 processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
-                this.getLogger().warn("No attributes found in the FlowFile to perform SNMP Set");
+                getLogger().warn("No attributes found in the FlowFile to perform SNMP Set");
             }
         }
     }
 
-    /**
-     * Method to construct {@link VariableBinding} based on {@link FlowFile}
-     * attributes in order to update the {@link PDU} that is going to be sent to
-     * the SNMP Agent.
-     * @param pdu {@link PDU} to be sent
-     * @param attributes {@link FlowFile} attributes
-     * @return true if at least one {@link VariableBinding} has been created, false otherwise
-     */
-    private boolean addVariables(PDU pdu, Map<String, String> attributes) {
-        boolean result = false;
-        for (Entry<String, String> attributeEntry : attributes.entrySet()) {
-            if (attributeEntry.getKey().startsWith(SNMPUtils.SNMP_PROP_PREFIX)) {
-                String[] splits = attributeEntry.getKey().split("\\" + SNMPUtils.SNMP_PROP_DELIMITER);
-                String snmpPropName = splits[1];
-                String snmpPropValue = attributeEntry.getValue();
-                if(SNMPUtils.OID_PATTERN.matcher(snmpPropName).matches()) {
-                    Variable var = null;
-                    if (splits.length == 2) { // no SMI syntax defined
-                        var = new OctetString(snmpPropValue);
-                    } else {
-                        int smiSyntax = Integer.valueOf(splits[2]);
-                        var = this.stringToVariable(snmpPropValue, smiSyntax);
-                    }
-                    if(var != null) {
-                        VariableBinding varBind = new VariableBinding(new OID(snmpPropName), var);
-                        pdu.add(varBind);
-                        result = true;
-                    }
-                }
+    private void processPdu(ProcessContext context, ProcessSession processSession, FlowFile flowFile, PDU pdu) {
+        try {
+            ResponseEvent response = snmpSetter.set(pdu);
+            if (response.getResponse() == null) {
+                processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
+                getLogger().error("Set request timed out or parameters are incorrect.");
+                context.yield();
+            } else if (response.getResponse().getErrorStatus() == PDU.noError) {
+                flowFile = SNMPUtils.updateFlowFileAttributesWithPduProperties(pdu, flowFile, processSession);
+                processSession.transfer(flowFile, REL_SUCCESS);
+                processSession.getProvenanceReporter().send(flowFile, snmpContext.getTarget().getAddress().toString());
+            } else {
+                final String error = response.getResponse().getErrorStatusText();
+                flowFile = SNMPUtils.addAttribute(SNMPUtils.SNMP_PROP_PREFIX + "error", error, flowFile, processSession);
+                processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
+                getLogger().error("Failed while executing SNMP Set [{}] via {}. Error = {}", response.getRequest().getVariableBindings(), snmpSetter, error);
             }
+        } catch (IOException e) {
+            processSession.transfer(processSession.penalize(flowFile), REL_FAILURE);
+            getLogger().error("Failed while executing SNMP Set via " + snmpSetter, e);

Review comment:
       Recommend replacing string concatenation with placeholder variable:
   ```suggestion
               getLogger().error("Failed while executing SNMP Set using {}", snmpSetter, e);
   ```

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/utils/SNMPUtils.java
##########
@@ -183,86 +192,150 @@ private static void addGetOidValues(Map<String, String> attributes, Object vecto
 
     /**
      * Method to add {@link FlowFile} attributes from a {@link VariableBinding}
+     *
      * @param variableBinding {@link VariableBinding}
-     * @param attributes {@link FlowFile} attributes to update
+     * @param attributes      {@link FlowFile} attributes to update
      */
     private static void addAttributeFromVariable(VariableBinding variableBinding, Map<String, String> attributes) {
         attributes.put(SNMP_PROP_PREFIX + variableBinding.getOid() + SNMP_PROP_DELIMITER + variableBinding.getVariable().getSyntax(), variableBinding.getVariable().toString());
     }
 
     /**
      * Will validate if provided name corresponds to valid SNMP property.
+     *
      * @param name the name of the property
      * @return 'true' if valid otherwise 'false'
      */
     public static boolean isValidSnmpPropertyName(String name) {
-        return propertyNames.contains(name);
+        return PROPERTY_NAMES.contains(name);
     }
 
     /**
      * Method to extract property name from given {@link Method}
+     *
      * @param method method
      * @return property name
      */
     private static String extractPropertyNameFromMethod(Method method) {
-        char c[] = method.getName().substring(3).toCharArray();
+        char[] c = method.getName().substring(3).toCharArray();
         c[0] = Character.toLowerCase(c[0]);
         return SNMP_PROP_PREFIX + new String(c);
     }
 
     /**
-     * Method to return the private protocol given the property
+     * Method to return the private protocol given the property.
+     *
      * @param privProtocol property
      * @return protocol
      */
     public static OID getPriv(String privProtocol) {
         switch (privProtocol) {
-        case "DES":
-            return PrivDES.ID;
-        case "3DES":
-            return Priv3DES.ID;
-        case "AES128":
-            return PrivAES128.ID;
-        case "AES192":
-            return PrivAES192.ID;
-        case "AES256":
-            return PrivAES256.ID;
-        default:
-            return null;
+            case "DES":

Review comment:
       As these values are also used as allowable property values, it would be helpful to define an enum that could be reused in both places.  The enum could also be used to include the associated OID which could remove the need for this method.

##########
File path: nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/java/org/apache/nifi/snmp/utils/SNMPUtils.java
##########
@@ -117,45 +122,48 @@ public static FlowFile updateFlowFileAttributesWithPduProperties(PDU response, F
                 }
                 flowFile = processSession.putAllAttributes(flowFile, attributes);
             } catch (Exception e) {
-                logger.warn("Failed to update FlowFile with AMQP attributes", e);
+                LOGGER.warn("Failed to update FlowFile with AMQP attributes", e);

Review comment:
       Although this was an existing message, should it reference SNMP instead of AMQP?
   ```suggestion
                   LOGGER.warn("Failed to update {} with SNMP attributes", flowFile, e);
   ```




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

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