You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by brianburnett <gi...@git.apache.org> on 2016/09/13 19:21:20 UTC

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

GitHub user brianburnett opened a pull request:

    https://github.com/apache/nifi/pull/1016

    NIFI-2724 New JMX Processor

    Attempt to contribute GetJMX Processor

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/brianburnett/nifi nifi-2724

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/1016.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1016
    
----
commit acdb3252ad330b829386e1b378eb013072cd4c15
Author: brianburnett <br...@capitalone.com>
Date:   2016-09-13T19:13:00Z

    NIFI-2724 New JMX Processor

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    I think that the commit is now amended correctly - possible to verify?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114884423
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    --- End diff --
    
    same at the end of this line, two spaces.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114890435
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    --- End diff --
    
    Instead of querying everything and then using whitelist and blacklist, wouldn't be easier to directly specify what we want when executing the query? I'd assume that users would either want everything or just a subset of known MBean names. That would simplify a lot the processor.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Dang - I have little git experience.  Know the best way to move forward from here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Got some outside assistance but believe I have the rebase done correctly now.  Could you verify - Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett looks like there has been good review/edit cycles here.  Can you rebase and push and advise when ready for a last look.  I can imagine a lot of folks will be pleased to have this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett You didn't respond to the feedback from @pvillard31. Do you need help getting this pushed through?


---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    No no. Cherry pick your commit number. 
    
    You will likely to fix conflicts in the process but  git will flag those files with conflict so once you fix it should be ok at the end


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114883476
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-nar/pom.xml ---
    @@ -0,0 +1,35 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  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.
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"><modelVersion>4.0.0</modelVersion>
    +    <parent>
    +        <groupId>org.apache.nifi</groupId>
    +        <artifactId>nifi-jmx-bundle</artifactId>
    +        <version>1.2.0-SNAPSHOT</version>
    +    </parent>
    +    <artifactId>nifi-jmx-nar</artifactId>
    +    <packaging>nar</packaging>
    +    <description>NiFi NAR for interacting with Java Managment Extensions</description>
    +    <properties>
    --- End diff --
    
    Are these properties required?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett can't you simply rebase?
    
    Cheers


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114888643
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record();
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                String type = (mbeanProperties.containsKey("type")) ? mbeanProperties.get("type") : "";
    +
    +                if (canProcess(domain, type)) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            logger.warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    logger.info("FILTERED: domain [" + domain + "] type [" + type + "]");
    +                }
    +            }
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            logger.error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          logger.error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            logger.error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            logger.error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < context.getProperty(BATCH_SIZE).asInteger()) {
    +            final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.SECONDS) * 1000;
    --- End diff --
    
    ````java
    final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
    ````


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    I believe I have the rebase in order - Could you please verify?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r87081671
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    --- End diff --
    
    Do you have a working example of using a static initializer?  I may be wrong but I believe the ProcessorInitializationContext is passed only in runtime.  Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Didn't your previous post. 
    
    git cherry-pick master?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114887372
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    --- End diff --
    
    whitespaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Try
    
    git checkout master -b nifi-2724v2
    git cherry-pick your commit hash
    git push - f origin nifi-2724
    
    This will create a temporary branch
    Allow you to copy a commit from another branch (cherry pick) and the force git to push to your original branch


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85256230
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        //descriptors.add(SSL_CONTEXT_SERVICE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record(jsonBuilderFactory);
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                if (mbeanProperties.containsKey("type") && canProcess(domain, mbeanProperties.get("type"))) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            getLogger().warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    getLogger().info("SKIPPING: domain [" + domain + "] type [" + mbeanProperties.get("type") + "]");
    +                }
    +            }
    +
    +            jmxc.close();
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            getLogger().error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          getLogger().error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            getLogger().error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            getLogger().error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < 100) {
    +            final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.SECONDS) * 1000;
    +
    +            if ((queueLastUpdated.get() < System.currentTimeMillis() - pollingMillis) && listingLock.tryLock()) {
    +                try {
    +                    final Set<Record> metrics = performQuery( context );
    +
    +                    queueLock.lock();
    +
    +                    try {
    +                        metrics.removeAll(inProcess);
    +                        metrics.removeAll(recentlyProcessed);
    +
    +                        queue.clear();
    +                        queue.addAll(metrics);
    +
    +                        queueLastUpdated.set(System.currentTimeMillis());
    +                        recentlyProcessed.clear();
    +
    +                        if (metrics.isEmpty()) {
    +                            context.yield();
    +                        }
    +                    } finally {
    +                        queueLock.unlock();
    +                    }
    +                } finally {
    +                    listingLock.unlock();
    +                }
    +            }
    +        }
    +
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final List<Record> jmxList = new ArrayList<>(batchSize);
    +        queueLock.lock();
    +        try {
    +            queue.drainTo(jmxList, batchSize);
    +            if (jmxList.isEmpty()) {
    +                return;
    +            } else {
    +                inProcess.addAll(jmxList);
    +            }
    +        } finally {
    +            queueLock.unlock();
    +        }
    +
    +        final ListIterator<Record> itr = jmxList.listIterator();
    +
    +        FlowFile flowFile = null;
    +        try {
    +            while (itr.hasNext()) {
    +                final Record record = itr.next();
    +
    +                flowFile = session.create();
    +                flowFile = session.write(flowFile,new OutputStreamCallback() {
    +                    @Override
    +                    public void process(final OutputStream out) throws IOException {
    +                        try {
    +                            out.write(record.toString().getBytes());
    +                        } catch (Exception e) {
    +                            getLogger().error("Exception writing metric record to flowfile: " + e.getMessage() +
    +                                    ": record content: [" + record.toString() + "]" );
    +                        }
    +                    }
    +                });
    +
    +                session.transfer(flowFile, REL_SUCCESS);
    --- End diff --
    
    Originally I had considered including some of the configuration properties as attributes in the flowFile and then ruled against it in favor of passing just the data and allowing the downstream processors to decorate it.
    
    We have been using this processor for a while now and I intended to reverse that decision since it would make our lives easier to just include it as it is generated.  I was waiting on your decision to apply the PR or not before making any changes.
    
    The JMX flowFile is basically MBean content stored in an array of properties and attributes.  It will be helpful to include the host, port, and timestamp and I was going to add them into the JSON object itself.  Is the best practice to inject these values into the payload or as attributes on the flowFile or both?
    
    Here is a JMX flowFile sample:
    {"domain":"kafka.cluster","properties":[{"key":"topic","value":"__consumer_offsets"},{"key":"name","value":"UnderReplicated"},{"key":"partition","value":"40"},{"key":"type","value":"Partition"}],"attributes":[{"key":"Value","value":"0"}]} 
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85244415
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        //descriptors.add(SSL_CONTEXT_SERVICE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record(jsonBuilderFactory);
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                if (mbeanProperties.containsKey("type") && canProcess(domain, mbeanProperties.get("type"))) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            getLogger().warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    getLogger().info("SKIPPING: domain [" + domain + "] type [" + mbeanProperties.get("type") + "]");
    +                }
    +            }
    +
    +            jmxc.close();
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            getLogger().error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          getLogger().error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            getLogger().error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            getLogger().error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < 100) {
    --- End diff --
    
    is there any reason this value is hardcoded?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @pvillard31 If he doesn't respond, I can take a stab at addressing the feedback so this can be closed out.


---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85259868
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        //descriptors.add(SSL_CONTEXT_SERVICE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record(jsonBuilderFactory);
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                if (mbeanProperties.containsKey("type") && canProcess(domain, mbeanProperties.get("type"))) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            getLogger().warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    getLogger().info("SKIPPING: domain [" + domain + "] type [" + mbeanProperties.get("type") + "]");
    +                }
    +            }
    +
    +            jmxc.close();
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            getLogger().error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          getLogger().error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            getLogger().error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            getLogger().error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < 100) {
    +            final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.SECONDS) * 1000;
    +
    +            if ((queueLastUpdated.get() < System.currentTimeMillis() - pollingMillis) && listingLock.tryLock()) {
    +                try {
    +                    final Set<Record> metrics = performQuery( context );
    +
    +                    queueLock.lock();
    +
    +                    try {
    +                        metrics.removeAll(inProcess);
    +                        metrics.removeAll(recentlyProcessed);
    +
    +                        queue.clear();
    +                        queue.addAll(metrics);
    +
    +                        queueLastUpdated.set(System.currentTimeMillis());
    +                        recentlyProcessed.clear();
    +
    +                        if (metrics.isEmpty()) {
    +                            context.yield();
    +                        }
    +                    } finally {
    +                        queueLock.unlock();
    +                    }
    +                } finally {
    +                    listingLock.unlock();
    +                }
    +            }
    +        }
    +
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final List<Record> jmxList = new ArrayList<>(batchSize);
    +        queueLock.lock();
    +        try {
    +            queue.drainTo(jmxList, batchSize);
    +            if (jmxList.isEmpty()) {
    +                return;
    +            } else {
    +                inProcess.addAll(jmxList);
    +            }
    +        } finally {
    +            queueLock.unlock();
    +        }
    +
    +        final ListIterator<Record> itr = jmxList.listIterator();
    +
    +        FlowFile flowFile = null;
    +        try {
    +            while (itr.hasNext()) {
    +                final Record record = itr.next();
    +
    +                flowFile = session.create();
    +                flowFile = session.write(flowFile,new OutputStreamCallback() {
    +                    @Override
    +                    public void process(final OutputStream out) throws IOException {
    +                        try {
    +                            out.write(record.toString().getBytes());
    +                        } catch (Exception e) {
    +                            getLogger().error("Exception writing metric record to flowfile: " + e.getMessage() +
    +                                    ": record content: [" + record.toString() + "]" );
    +                        }
    +                    }
    +                });
    +
    +                session.transfer(flowFile, REL_SUCCESS);
    --- End diff --
    
    I usually assume that it depends on the design chosen, how useful they are for downstream processors, the total number of attributes and more importantly, the size/lenght of the attributes.
    
    If you are likely to use it for routing the flowfile, than I reckon they can be added directly, instead of requiring the use to read a large JSON (i.e. payload * BATCH_SIZE), which I suspect would happen in memory.
    
    ListenSyslog in batching mode should be a good example of a batched + attributes processor.
    
    Cheers


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114884654
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    --- End diff --
    
    I am not sure to understand the role of this property? Isn't it already handled by the scheduling of the processor itself? Or do you expect some kind of additional scheduling on top of it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114889131
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/ListFilter.java ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.processors.jmx;
    +
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class ListFilter {
    +    private String cachedListString = "";
    +    private Map<String,ArrayList<String>> listM = null;
    +
    +    public static final boolean WHITELIST = true;
    +    public static final boolean BLACKLIST = false;
    +
    +
    +    public ListFilter( String listString ) {
    +        setListString( listString );
    +    }
    +
    +
    +    public void setListString( String listString ) {
    +        if (listString == null || listString.isEmpty() ) listString = "";
    +
    +        if (!listString.equals(cachedListString) || listM == null) {
    +            buildList( listString );
    +        }
    +    }
    +
    +
    +    private void buildList(String listString) {
    +        listM = new HashMap<>();
    +
    +        if (!listString.isEmpty()) {
    +            String[] listElementsAr = listString.replaceAll("\\s+"," ").split(",");
    +
    +            for (int i = 0; i < listElementsAr.length; i++) {
    +                String[] listPartsAr = listElementsAr[i].split(":");
    +                ArrayList<String> listTypesAL = new ArrayList<String>();
    +
    +                if (listPartsAr.length > 1) {
    +                    String[] listTypeElementsAr = listPartsAr[1].split(" ");
    +
    +                    for (String s : listTypeElementsAr) {
    +                        if (!s.replaceAll("\\s+","").isEmpty()) {
    +                            listTypesAL.add(s);
    +                        }
    +                    }
    +                }
    +
    +                listM.put(listPartsAr[0], listTypesAL);
    +            }
    +        }
    +
    +        cachedListString = listString;
    +    }
    +
    +
    +
    +    /*
    +     * domain and type values can contain regular expressions
    +     * First check if the domain key in matched in the Map.  If yes then check if the type
    +     * is matched in the ArrayList pointed to by the domain key.
    +     * Is in list if...
    +     * domain key is not matched: false
    +     * domain key is matched and there are no entries in the ArrayList: true
    +     * domain key is matched and there are entries in the ArrayList but type value is not matched: false
    +     * domain key is matched and there are entries in the ArrayList and the type value is matched: true
    +     */
    +    public boolean isInList(boolean listStyle, String domain, String type) {
    +        // not in the list...
    +        // whitelist = true since not defining a value for this style means the everything is included
    +        // blacklist = false since not defining a value for this style means nothing is excluded
    +        if (listM.size() == 0) {
    +            return (listStyle) ? true : false;
    +        }
    +
    +        if (domain == null || domain.isEmpty()) {
    +            return (listStyle) ? true : false;
    +        }
    +
    +        if (type == null) type = "";
    --- End diff --
    
    Please avoid single line if statement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    $git log
    commit df73bdc7f0f13d4f21b23a8a0a41f78fa0386aa2
    Author: brianburnett <br...@capitalone.com>
    Date:   Thu Feb 16 11:55:55 2017 -0500
    
        NIFI-2724 JMX Processor 1.2.0 rev
    
    $git cherry-pick df73bdc7f0f13d4f21b23a8a0a41f78fa0386aa2
    On branch nifi-2724v2
    You are currently cherry-picking commit df73bdc.
    
    nothing to commit, working tree clean
    The previous cherry-pick is now empty, possibly due to conflict resolution.
    If you wish to commit it anyway, use:
    
        git commit --allow-empty
    
    Otherwise, please use 'git reset'
    
    
    Shall I do the git commit --allow-empty?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by automaticgiant <gi...@git.apache.org>.
Github user automaticgiant commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @trixpan cherry-pick complete. what say you?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85245213
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        //descriptors.add(SSL_CONTEXT_SERVICE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record(jsonBuilderFactory);
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                if (mbeanProperties.containsKey("type") && canProcess(domain, mbeanProperties.get("type"))) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            getLogger().warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    getLogger().info("SKIPPING: domain [" + domain + "] type [" + mbeanProperties.get("type") + "]");
    +                }
    +            }
    +
    +            jmxc.close();
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            getLogger().error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          getLogger().error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            getLogger().error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            getLogger().error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < 100) {
    +            final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.SECONDS) * 1000;
    +
    +            if ((queueLastUpdated.get() < System.currentTimeMillis() - pollingMillis) && listingLock.tryLock()) {
    +                try {
    +                    final Set<Record> metrics = performQuery( context );
    +
    +                    queueLock.lock();
    +
    +                    try {
    +                        metrics.removeAll(inProcess);
    +                        metrics.removeAll(recentlyProcessed);
    +
    +                        queue.clear();
    +                        queue.addAll(metrics);
    +
    +                        queueLastUpdated.set(System.currentTimeMillis());
    +                        recentlyProcessed.clear();
    +
    +                        if (metrics.isEmpty()) {
    +                            context.yield();
    +                        }
    +                    } finally {
    +                        queueLock.unlock();
    +                    }
    +                } finally {
    +                    listingLock.unlock();
    +                }
    +            }
    +        }
    +
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final List<Record> jmxList = new ArrayList<>(batchSize);
    +        queueLock.lock();
    +        try {
    +            queue.drainTo(jmxList, batchSize);
    +            if (jmxList.isEmpty()) {
    +                return;
    +            } else {
    +                inProcess.addAll(jmxList);
    +            }
    +        } finally {
    +            queueLock.unlock();
    +        }
    +
    +        final ListIterator<Record> itr = jmxList.listIterator();
    +
    +        FlowFile flowFile = null;
    +        try {
    +            while (itr.hasNext()) {
    +                final Record record = itr.next();
    +
    +                flowFile = session.create();
    +                flowFile = session.write(flowFile,new OutputStreamCallback() {
    +                    @Override
    +                    public void process(final OutputStream out) throws IOException {
    +                        try {
    +                            out.write(record.toString().getBytes());
    +                        } catch (Exception e) {
    +                            getLogger().error("Exception writing metric record to flowfile: " + e.getMessage() +
    +                                    ": record content: [" + record.toString() + "]" );
    +                        }
    +                    }
    +                });
    +
    +                session.transfer(flowFile, REL_SUCCESS);
    --- End diff --
    
    I noticed your data has no attributes at all.
    
    Is there anything coming from JMX that may be useful to the user? (e.g. source of the data)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r103875643
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,415 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    --- End diff --
    
    If this is a getter how would you be using the failure relationship? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett there's a minor conflict but I am more concerned with consistent failures when I run the test units?
    
    Have you been able to build this with `-Pcontrib-check` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85252576
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    --- End diff --
    
    Ok - will add that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r88592140
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    --- End diff --
    
    Thanks! I looked at a couple of processors and it seems to make sense.  Would you like for me to make the changes in this PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Seems fixed. Will start reviewing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett seems to be better now. 
    
    I also suggest you run $ git commit --amend and remove the
    
    ```
    # The first commit's message is:
    ```
    
    and 
    
    ```
    # This is the 2nd commit message:
    
    NIFI-2724 fix processor version
    ```
    
    So that the commit message reads as:
    
    `NIFI-2724 New JMX Processor`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114885148
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    --- End diff --
    
    Following my previous comment, if you want to define a default value of 300 seconds for the processor scheduling strategy, you can add the following annotation to the class:
    ````java
    @DefaultSchedule(strategy = SchedulingStrategy.TIMER_DRIVEN, period = "300 sec")
    ````


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114888247
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record();
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                String type = (mbeanProperties.containsKey("type")) ? mbeanProperties.get("type") : "";
    +
    +                if (canProcess(domain, type)) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            logger.warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    logger.info("FILTERED: domain [" + domain + "] type [" + type + "]");
    +                }
    +            }
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            logger.error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          logger.error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            logger.error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            logger.error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < context.getProperty(BATCH_SIZE).asInteger()) {
    +            final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.SECONDS) * 1000;
    +
    +            if ((queueLastUpdated.get() < System.currentTimeMillis() - pollingMillis) && listingLock.tryLock()) {
    +                try {
    +                    final Set<Record> metrics = performQuery( context );
    +
    +                    queueLock.lock();
    +
    +                    try {
    +                        metrics.removeAll(inProcess);
    +                        metrics.removeAll(recentlyProcessed);
    +
    +                        queue.clear();
    +                        queue.addAll(metrics);
    +
    +                        queueLastUpdated.set(System.currentTimeMillis());
    +                        recentlyProcessed.clear();
    +
    +                        if (metrics.isEmpty()) {
    +                            context.yield();
    +                        }
    +                    } finally {
    +                        queueLock.unlock();
    +                    }
    +                } finally {
    +                    listingLock.unlock();
    +                }
    +            }
    +        }
    +
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final List<Record> jmxList = new ArrayList<>(batchSize);
    +        queueLock.lock();
    +        try {
    +            queue.drainTo(jmxList, batchSize);
    +            if (jmxList.isEmpty()) {
    +                return;
    +            } else {
    +                inProcess.addAll(jmxList);
    +            }
    +        } finally {
    +            queueLock.unlock();
    +        }
    +
    +        final ListIterator<Record> itr = jmxList.listIterator();
    +
    +        FlowFile flowFile = null;
    +        try {
    +            while (itr.hasNext()) {
    +                final Record record = itr.next();
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final Timestamp timestamp = new Timestamp(System.currentTimeMillis());
    +
    +                attributes.put(HOSTNAME_ATTRIBUTE, context.getProperty(HOSTNAME).getValue());
    +                attributes.put(PORT_ATTRIBUTE, context.getProperty(PORT).getValue());
    +                attributes.put(TIMESTAMP_ATTRIBUTE, timestamp.toString());
    +
    +                flowFile = session.create();
    +                flowFile = session.putAllAttributes(flowFile, attributes);
    +                logger.info("ATTRIBUTES: " + context.getProperty(HOSTNAME).getValue() + " " +
    +                        context.getProperty(PORT).getValue() + " " + timestamp.toString());
    +
    +                flowFile = session.write(flowFile,new OutputStreamCallback() {
    +                    @Override
    +                    public void process(final OutputStream out) throws IOException {
    +                        try {
    +                            out.write(record.toJsonString().getBytes());
    +                        } catch (Exception e) {
    +                            logger.error("Exception writing metric record to flowfile: " + e.getMessage() +
    +                                    ": record content: [" + record.toJsonString() + "]" );
    +                        }
    +                    }
    +                });
    +
    +                session.getProvenanceReporter().create(flowFile);
    +
    +                session.transfer(flowFile, REL_SUCCESS);
    +                logger.info("Added {} to flow", new Object[]{flowFile});
    +
    +                if (!isScheduled()) {  // if processor stopped, put the rest of the files back on the queue.
    +                    queueLock.lock();
    +                    try {
    +                        while (itr.hasNext()) {
    +                            final Record nextRecord = itr.next();
    +                            queue.add(nextRecord);
    +                            inProcess.remove(nextRecord);
    +                        }
    +                    } finally {
    +                        queueLock.unlock();
    +                    }
    +                }
    +            }
    +            session.commit();
    +        } catch (final Exception e) {
    +            logger.error( "Exception fetching records: " + e.getMessage() );
    +
    +            if (flowFile != null) {
    +                session.remove(flowFile);
    +            }
    +        } finally {
    +            queueLock.lock();
    +            try {
    +                inProcess.removeAll(jmxList);
    +                recentlyProcessed.addAll(jmxList);
    +            } finally {
    +                queueLock.unlock();
    +            }
    +        }
    +    }
    +
    +    private boolean canProcess( String domain, String type ) {
    +        if (blackListFilter.isInList(ListFilter.BLACKLIST, domain, type )) {
    +            return false;
    +        }
    +
    +        return (whiteListFilter.isInList(ListFilter.WHITELIST, domain, type )) ? true : false;
    --- End diff --
    
    Just return the boolean:
    ````java
    return whiteListFilter.isInList(ListFilter.WHITELIST, domain, type );
    ````


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Everything looks ok on my side but I see another merge conflict.  Top down compiled with the contrib-check. Possible to get detailed instructions on what I need to do?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114884244
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    --- End diff --
    
    Two spaces instead of one "type(s).  Domains"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114888321
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record();
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                String type = (mbeanProperties.containsKey("type")) ? mbeanProperties.get("type") : "";
    +
    +                if (canProcess(domain, type)) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            logger.warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    logger.info("FILTERED: domain [" + domain + "] type [" + type + "]");
    +                }
    +            }
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            logger.error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          logger.error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            logger.error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            logger.error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < context.getProperty(BATCH_SIZE).asInteger()) {
    +            final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.SECONDS) * 1000;
    +
    +            if ((queueLastUpdated.get() < System.currentTimeMillis() - pollingMillis) && listingLock.tryLock()) {
    +                try {
    +                    final Set<Record> metrics = performQuery( context );
    +
    +                    queueLock.lock();
    +
    +                    try {
    +                        metrics.removeAll(inProcess);
    +                        metrics.removeAll(recentlyProcessed);
    +
    +                        queue.clear();
    +                        queue.addAll(metrics);
    +
    +                        queueLastUpdated.set(System.currentTimeMillis());
    +                        recentlyProcessed.clear();
    +
    +                        if (metrics.isEmpty()) {
    +                            context.yield();
    +                        }
    +                    } finally {
    +                        queueLock.unlock();
    +                    }
    +                } finally {
    +                    listingLock.unlock();
    +                }
    +            }
    +        }
    +
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final List<Record> jmxList = new ArrayList<>(batchSize);
    +        queueLock.lock();
    +        try {
    +            queue.drainTo(jmxList, batchSize);
    +            if (jmxList.isEmpty()) {
    +                return;
    +            } else {
    +                inProcess.addAll(jmxList);
    +            }
    +        } finally {
    +            queueLock.unlock();
    +        }
    +
    +        final ListIterator<Record> itr = jmxList.listIterator();
    +
    +        FlowFile flowFile = null;
    +        try {
    +            while (itr.hasNext()) {
    +                final Record record = itr.next();
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final Timestamp timestamp = new Timestamp(System.currentTimeMillis());
    +
    +                attributes.put(HOSTNAME_ATTRIBUTE, context.getProperty(HOSTNAME).getValue());
    +                attributes.put(PORT_ATTRIBUTE, context.getProperty(PORT).getValue());
    +                attributes.put(TIMESTAMP_ATTRIBUTE, timestamp.toString());
    +
    +                flowFile = session.create();
    +                flowFile = session.putAllAttributes(flowFile, attributes);
    +                logger.info("ATTRIBUTES: " + context.getProperty(HOSTNAME).getValue() + " " +
    +                        context.getProperty(PORT).getValue() + " " + timestamp.toString());
    +
    +                flowFile = session.write(flowFile,new OutputStreamCallback() {
    +                    @Override
    +                    public void process(final OutputStream out) throws IOException {
    +                        try {
    +                            out.write(record.toJsonString().getBytes());
    +                        } catch (Exception e) {
    +                            logger.error("Exception writing metric record to flowfile: " + e.getMessage() +
    +                                    ": record content: [" + record.toJsonString() + "]" );
    +                        }
    +                    }
    +                });
    +
    +                session.getProvenanceReporter().create(flowFile);
    +
    +                session.transfer(flowFile, REL_SUCCESS);
    +                logger.info("Added {} to flow", new Object[]{flowFile});
    --- End diff --
    
    I'd consider debug level instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett 
    
    I suspect the reason for that is because you are trying to cherry-pick from the branch you already have a commit.
    
    Note how I suggested you to circle around...
    
    First step is to create a new branch from master. If you master already has the commit then you can use the upstream/master as a base (assuming you gave the github.com/apache/nifi remote the name of upstream)
    
    `git checkout origin master -b nifi-2724v2`
    
    If you do `git log` you should see the last commit as being whatever has been just published to the NiFi master git and also be told by `git status` that you are using nifi-2724v2`
    
    Now you perform a cherry pick that will copy your changes from the old nifi-2742 branch into to the branch (i.e. nifi-2724v2) you just created.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85254162
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        //descriptors.add(SSL_CONTEXT_SERVICE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record(jsonBuilderFactory);
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                if (mbeanProperties.containsKey("type") && canProcess(domain, mbeanProperties.get("type"))) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            getLogger().warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    getLogger().info("SKIPPING: domain [" + domain + "] type [" + mbeanProperties.get("type") + "]");
    +                }
    +            }
    +
    +            jmxc.close();
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            getLogger().error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          getLogger().error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            getLogger().error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            getLogger().error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < 100) {
    --- End diff --
    
    I modeled after the GetFile processor which still appears to use a hard coded value.  I can use the property value here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114886280
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/ListFilter.java ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.processors.jmx;
    +
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class ListFilter {
    +    private String cachedListString = "";
    +    private Map<String,ArrayList<String>> listM = null;
    +
    +    public static final boolean WHITELIST = true;
    +    public static final boolean BLACKLIST = false;
    +
    +
    +    public ListFilter( String listString ) {
    +        setListString( listString );
    +    }
    +
    +
    +    public void setListString( String listString ) {
    +        if (listString == null || listString.isEmpty() ) listString = "";
    --- End diff --
    
    Why are you checking listString.isEmpty() to set it empty after? Am I missing something?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114887336
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    --- End diff --
    
    Would directly use final types:
    ````java
    String hostname = context.getProperty(HOSTNAME).getValue();
    int context.getProperty(PORT).asInteger();
    String context.getProperty(WHITELIST).getValue();
    String blacklist = context.getProperty(BLACKLIST).getValue();
    ````


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114888418
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record();
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                String type = (mbeanProperties.containsKey("type")) ? mbeanProperties.get("type") : "";
    +
    +                if (canProcess(domain, type)) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            logger.warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    logger.info("FILTERED: domain [" + domain + "] type [" + type + "]");
    +                }
    +            }
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            logger.error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          logger.error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            logger.error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            logger.error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < context.getProperty(BATCH_SIZE).asInteger()) {
    +            final long pollingMillis = context.getProperty(POLLING_INTERVAL).asTimePeriod(TimeUnit.SECONDS) * 1000;
    +
    +            if ((queueLastUpdated.get() < System.currentTimeMillis() - pollingMillis) && listingLock.tryLock()) {
    +                try {
    +                    final Set<Record> metrics = performQuery( context );
    +
    +                    queueLock.lock();
    +
    +                    try {
    +                        metrics.removeAll(inProcess);
    +                        metrics.removeAll(recentlyProcessed);
    +
    +                        queue.clear();
    +                        queue.addAll(metrics);
    +
    +                        queueLastUpdated.set(System.currentTimeMillis());
    +                        recentlyProcessed.clear();
    +
    +                        if (metrics.isEmpty()) {
    +                            context.yield();
    +                        }
    +                    } finally {
    +                        queueLock.unlock();
    +                    }
    +                } finally {
    +                    listingLock.unlock();
    +                }
    +            }
    +        }
    +
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final List<Record> jmxList = new ArrayList<>(batchSize);
    +        queueLock.lock();
    +        try {
    +            queue.drainTo(jmxList, batchSize);
    +            if (jmxList.isEmpty()) {
    +                return;
    +            } else {
    +                inProcess.addAll(jmxList);
    +            }
    +        } finally {
    +            queueLock.unlock();
    +        }
    +
    +        final ListIterator<Record> itr = jmxList.listIterator();
    +
    +        FlowFile flowFile = null;
    +        try {
    +            while (itr.hasNext()) {
    +                final Record record = itr.next();
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final Timestamp timestamp = new Timestamp(System.currentTimeMillis());
    +
    +                attributes.put(HOSTNAME_ATTRIBUTE, context.getProperty(HOSTNAME).getValue());
    +                attributes.put(PORT_ATTRIBUTE, context.getProperty(PORT).getValue());
    +                attributes.put(TIMESTAMP_ATTRIBUTE, timestamp.toString());
    +
    +                flowFile = session.create();
    +                flowFile = session.putAllAttributes(flowFile, attributes);
    +                logger.info("ATTRIBUTES: " + context.getProperty(HOSTNAME).getValue() + " " +
    --- End diff --
    
    I'd consider debug level here, otherwise, could generate quite a lot of logs based on how the processor is configured.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by jdye64 <gi...@git.apache.org>.
Github user jdye64 commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett once you make the recommended changes I can help you with the rebasing effort.


---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114887973
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record();
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                String type = (mbeanProperties.containsKey("type")) ? mbeanProperties.get("type") : "";
    +
    +                if (canProcess(domain, type)) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            logger.warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    logger.info("FILTERED: domain [" + domain + "] type [" + type + "]");
    --- End diff --
    
    Would probably consider debug level here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85258090
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    --- End diff --
    
    yes please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Looks good. Will review 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    hey @brianburnett thank you for submiting a new commit. 
    
    You seem to have merged the mater into your PR? You should have instead rebased the PR to master.
    
    Please refer to the following comments to revisit how to fix it.
    
    https://github.com/apache/nifi/pull/1016#issuecomment-280458891
    
    Cheers!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114887820
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +@WritesAttributes({
    +        @WritesAttribute(attribute="hostname", description="The name of the host that the object originates."),
    +        @WritesAttribute(attribute="port", description="The JMX connection port that the object originates."),
    +        @WritesAttribute(attribute="timestamp", description="The timestamp of when the object was emitted.") })
    +
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .displayName("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .displayName("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .displayName("Domain White List")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .displayName("Domain Black List")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor
    +            .Builder().name("PollingInterval")
    +            .displayName("Polling Interval")
    +            .description("Indicates how long to wait before performing a connection to the RMI Server")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor
    +            .Builder().name("BatchSize")
    +            .displayName("Batch Size")
    +            .description("The maximum number of MBean records to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final String HOSTNAME_ATTRIBUTE = "hostname";
    +    public static final String PORT_ATTRIBUTE = "port";
    +    public static final String TIMESTAMP_ATTRIBUTE = "timestamp";
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    private final ComponentLog logger = getLogger();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record();
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                String type = (mbeanProperties.containsKey("type")) ? mbeanProperties.get("type") : "";
    +
    +                if (canProcess(domain, type)) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            logger.warn("Exception fetching MBean attribute: " + e.getMessage() +
    --- End diff --
    
    please add the exception as argument of the log message (same comment each time you're logging a message when catching an exception, it's really helpful to have more than just the message)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Rebased - jumped through some git hoops but I believe it is current.  Let me know. Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett would you be able to rebase and squash this PR? It currently includes other people commits that already have been merged to the master codebase


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r103875149
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/pom.xml ---
    @@ -0,0 +1,67 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!-- 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. -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <parent>
    +        <groupId>org.apache.nifi</groupId>
    +        <artifactId>nifi-jmx-bundle</artifactId>
    +        <version>1.2.0-SNAPSHOT</version>
    +    </parent>
    +    <modelVersion>4.0.0</modelVersion>
    +    <artifactId>nifi-jmx-processors</artifactId>
    +    <packaging>jar</packaging>
    +    <dependencies>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-api</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-processor-utils</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-utils</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.glassfish</groupId>
    +            <artifactId>javax.json</artifactId>
    +            <version>1.0.4</version>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-mock</artifactId>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>commons-io</groupId>
    +            <artifactId>commons-io</artifactId>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.slf4j</groupId>
    +            <artifactId>slf4j-simple</artifactId>
    +            <scope>test</scope>
    +        </dependency>
    +    </dependencies>
    +    <build>
    +        <plugins>
    +            <plugin>
    +                <groupId>org.apache.rat</groupId>
    +                <artifactId>apache-rat-plugin</artifactId>
    --- End diff --
    
    RAT should be inherited from the project pom


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114883768
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/pom.xml ---
    @@ -0,0 +1,55 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!-- 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. -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <parent>
    +        <groupId>org.apache.nifi</groupId>
    +        <artifactId>nifi-jmx-bundle</artifactId>
    +        <version>1.2.0-SNAPSHOT</version>
    +    </parent>
    +    <modelVersion>4.0.0</modelVersion>
    +    <artifactId>nifi-jmx-processors</artifactId>
    +    <packaging>jar</packaging>
    +    <dependencies>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-api</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-processor-utils</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-utils</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>com.google.code.gson</groupId>
    --- End diff --
    
    can you reference the version we've already defined in the parent pom? (version 2.7)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85259465
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(HOSTNAME);
    +        descriptors.add(PORT);
    +        descriptors.add(WHITELIST);
    +        descriptors.add(BLACKLIST);
    +        descriptors.add(POLLING_INTERVAL);
    +        descriptors.add(BATCH_SIZE);
    +        //descriptors.add(SSL_CONTEXT_SERVICE);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +        whiteListFilter = new ListFilter( "" );
    +        blackListFilter = new ListFilter( "" );
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        queue.clear();
    +    }
    +
    +    private Set<Record> performQuery( ProcessContext context) throws ProcessException {
    +        PropertyValue hostname = context.getProperty(HOSTNAME);
    +        PropertyValue port = context.getProperty(PORT);
    +        PropertyValue whitelist = context.getProperty(WHITELIST);
    +        PropertyValue blacklist = context.getProperty(BLACKLIST);
    +
    +        Set<Record> metricsSet = new HashSet<Record>();
    +
    +        whiteListFilter.setListString(whitelist.getValue());
    +        blackListFilter.setListString( blacklist.getValue() );
    +
    +        try {
    +            HashMap<String,Object> env = new HashMap<String,Object>();
    +
    +            StringBuilder urlStr = new StringBuilder("service:jmx:rmi:///jndi/rmi://");
    +            urlStr.append(hostname).append(":");
    +            urlStr.append(port);
    +            urlStr.append("/jmxrmi");
    +
    +            JMXServiceURL url = new JMXServiceURL(urlStr.toString());
    +            JMXConnector jmxc = JMXConnectorFactory.connect(url, null);
    +
    +            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
    +
    +            Set<ObjectName> mbeans = mbsc.queryNames(null, null);
    +            for (ObjectName mbean : mbeans) {
    +                String domain = mbean.getDomain();
    +
    +                Record record = new Record(jsonBuilderFactory);
    +                Hashtable<String, String> mbeanProperties = mbean.getKeyPropertyList();
    +
    +                if (mbeanProperties.containsKey("type") && canProcess(domain, mbeanProperties.get("type"))) {
    +                    record.setDomain( domain );
    +                    record.setProperties(mbeanProperties.entrySet());
    +
    +                    MBeanInfo mbeanInfo = mbsc.getMBeanInfo(mbean);
    +                    MBeanAttributeInfo[] mbeanAttributeInfoAr = mbeanInfo.getAttributes();
    +
    +                    boolean validRecord = true;
    +
    +                    Map<String, String> mbeanStringMap = new HashMap<String, String>();
    +
    +                    for (MBeanAttributeInfo mbeanAttributeInfo : mbeanAttributeInfoAr) {
    +                        try {
    +                            Object value = mbsc.getAttribute(mbean, mbeanAttributeInfo.getName());
    +
    +                            mbeanStringMap.put( mbeanAttributeInfo.getName(), value.toString() );
    +                        } catch (Exception e) {
    +                            // IF ERROR DO NOT ADD to metricsSet
    +                            validRecord = false;
    +                            getLogger().warn("Exception fetching MBean attribute: " + e.getMessage() +
    +                                    ": details: [" + mbean.getCanonicalName() + "]");
    +                        }
    +                    }
    +
    +                    if (validRecord) {
    +                        record.setAttributes(mbeanStringMap);
    +                        metricsSet.add(record);
    +                    }
    +                } else {
    +                    getLogger().info("SKIPPING: domain [" + domain + "] type [" + mbeanProperties.get("type") + "]");
    +                }
    +            }
    +
    +            jmxc.close();
    +        } catch( ProcessException pe ) {
    +            throw pe;
    +        } catch( IOException ioe) {
    +            getLogger().error( "Exception connecting to JMX RMI Listener: " + ioe.getMessage() + ": hostname [" +
    +                    hostname + "] port [" + port + "]");
    +        } catch( SecurityException se ) {
    +          getLogger().error( "Exception connecting to JMX RMI Listener due to security issues: " +
    +                    se.getMessage() + ": hostname [" + hostname + "] port [" + port + "]" );
    +        } catch( InstanceNotFoundException|IntrospectionException|ReflectionException e ) {
    +            getLogger().error( "Exception with MBean Server: " + e.getMessage() );
    +        } catch( Exception e ) {
    +            getLogger().error( "Exception performing MBean Query: " + e.getMessage() );
    +        }
    +
    +        return metricsSet;
    +    }
    +
    +
    +
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (queue.size() < 100) {
    --- End diff --
    
    I see. I would still avoid hardcoded values.
    
    Also, from what I understand the 100 will interact with the `BATCH_SIZE `property. Haven't tested but I suspect a `BATCH_SIZE `larger than 100 would be bogus?
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Try
    
    git checkout master -b nifi-2724v2
    git cherry-pick <commit>
    git push origin nifi-2724


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    It was my intent to finish pushing this through.  I can make the recommended changes but may need some assistance rebasing to the current version.


---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    The previous commit introduced a bug which I corrected with this last commit.  There is now a conflict in the nifi-nar-bundles pom with websocket and jmx bundle modules.  Just curious how you prefer that I correct this. -- Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114889040
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/ListFilter.java ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.processors.jmx;
    +
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class ListFilter {
    +    private String cachedListString = "";
    +    private Map<String,ArrayList<String>> listM = null;
    +
    +    public static final boolean WHITELIST = true;
    +    public static final boolean BLACKLIST = false;
    +
    +
    +    public ListFilter( String listString ) {
    +        setListString( listString );
    +    }
    +
    +
    +    public void setListString( String listString ) {
    +        if (listString == null || listString.isEmpty() ) listString = "";
    +
    +        if (!listString.equals(cachedListString) || listM == null) {
    +            buildList( listString );
    +        }
    +    }
    +
    +
    +    private void buildList(String listString) {
    +        listM = new HashMap<>();
    +
    +        if (!listString.isEmpty()) {
    +            String[] listElementsAr = listString.replaceAll("\\s+"," ").split(",");
    +
    +            for (int i = 0; i < listElementsAr.length; i++) {
    +                String[] listPartsAr = listElementsAr[i].split(":");
    +                ArrayList<String> listTypesAL = new ArrayList<String>();
    +
    +                if (listPartsAr.length > 1) {
    +                    String[] listTypeElementsAr = listPartsAr[1].split(" ");
    +
    +                    for (String s : listTypeElementsAr) {
    +                        if (!s.replaceAll("\\s+","").isEmpty()) {
    +                            listTypesAL.add(s);
    +                        }
    +                    }
    +                }
    +
    +                listM.put(listPartsAr[0], listTypesAL);
    +            }
    +        }
    +
    +        cachedListString = listString;
    +    }
    +
    +
    +
    +    /*
    +     * domain and type values can contain regular expressions
    +     * First check if the domain key in matched in the Map.  If yes then check if the type
    +     * is matched in the ArrayList pointed to by the domain key.
    +     * Is in list if...
    +     * domain key is not matched: false
    +     * domain key is matched and there are no entries in the ArrayList: true
    +     * domain key is matched and there are entries in the ArrayList but type value is not matched: false
    +     * domain key is matched and there are entries in the ArrayList and the type value is matched: true
    +     */
    +    public boolean isInList(boolean listStyle, String domain, String type) {
    +        // not in the list...
    +        // whitelist = true since not defining a value for this style means the everything is included
    +        // blacklist = false since not defining a value for this style means nothing is excluded
    +        if (listM.size() == 0) {
    +            return (listStyle) ? true : false;
    --- End diff --
    
    ````java
    return listStyle;
    ````
    
    Same comments for next part: it's useless to do: return boolean ? true : false
    You can just return boolean.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r87196410
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    --- End diff --
    
    @brianburnett many of our newer processors have that logic changed. The few you can look at are: PublishMQTT, AbstractSNMP as well as Email, JMS, AMQP, Kafka etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r103875353
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/pom.xml ---
    @@ -0,0 +1,67 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!-- 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. -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <parent>
    +        <groupId>org.apache.nifi</groupId>
    +        <artifactId>nifi-jmx-bundle</artifactId>
    +        <version>1.2.0-SNAPSHOT</version>
    +    </parent>
    +    <modelVersion>4.0.0</modelVersion>
    +    <artifactId>nifi-jmx-processors</artifactId>
    +    <packaging>jar</packaging>
    +    <dependencies>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-api</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-processor-utils</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-utils</artifactId>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.glassfish</groupId>
    +            <artifactId>javax.json</artifactId>
    --- End diff --
    
    Is there any chance you can use gson or any other gson JAR already used by other nifi processors? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r114883939
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,406 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Timestamp;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    --- End diff --
    
    To remove if no other processor is referenced here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    That was a merge, not a rebase :-(
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85244266
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    +            .Builder().name("Hostname")
    +            .description("The JMX Hostname or IP address")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PORT = new PropertyDescriptor
    +            .Builder().name("Port")
    +            .description("The JMX Port")
    +            .required(true)
    +            .addValidator(StandardValidators.PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor WHITELIST = new PropertyDescriptor
    +            .Builder().name("DomainWhiteList")
    +            .description("Include only these MBean domain(s) and type(s).  Domains are comma delimited "
    +                    + "and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]]")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor BLACKLIST = new PropertyDescriptor
    +            .Builder().name("DomainBlackList")
    +            .description("Include everything excluding these MBean domain(s) and type(s).   Domains are "
    +                    + "comma delimited and optional MBean types follow a colon and are space delimited.   "
    +                    + "Format: [domain1[:type1[ type2]][,domain2]])")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Polling Interval")
    +            .description("Indicates how long to wait before performing a directory listing")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .defaultValue("300 sec")
    +            .build();
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The maximum number of files to pull in each iteration")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("All FlowFiles that are created are routed to this relationship")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that encounter errors are routed to this relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +    private final BlockingQueue<Record> queue = new LinkedBlockingQueue<>();
    +
    +    private final Set<Record> inProcess = new HashSet<>();    // guarded by queueLock
    +    private final Set<Record> recentlyProcessed = new HashSet<>();    // guarded by queueLock
    +
    +    private final Lock queueLock = new ReentrantLock();
    +    private final Lock listingLock = new ReentrantLock();
    +    private final AtomicLong queueLastUpdated = new AtomicLong(0L);
    +
    +    private final JsonBuilderFactory jsonBuilderFactory = Json.createBuilderFactory(null);
    +
    +    private ListFilter whiteListFilter;
    +    private ListFilter blackListFilter;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    --- End diff --
    
    @olegz once gave me a hint that there's an issue with it since init() method as well as few others are invoked more then once per life-cycle of a processor (see https://issues.apache.org/jira/browse/NIFI-1318).
    
    Consider using a static initializer


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1016#discussion_r85244032
  
    --- Diff: nifi-nar-bundles/nifi-jmx-bundle/nifi-jmx-processors/src/main/java/org/apache/nifi/processors/jmx/GetJMX.java ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.processors.jmx;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +
    +import javax.json.Json;
    +import javax.json.JsonBuilderFactory;
    +import javax.management.MBeanAttributeInfo;
    +import javax.management.MBeanInfo;
    +import javax.management.MBeanServerConnection;
    +import javax.management.ObjectName;
    +import javax.management.remote.JMXConnector;
    +import javax.management.remote.JMXConnectorFactory;
    +import javax.management.remote.JMXServiceURL;
    +import javax.management.ReflectionException;
    +import javax.management.InstanceNotFoundException;
    +import javax.management.IntrospectionException;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.Map;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Hashtable;
    +import java.util.ListIterator;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +@Tags({"JMX"})
    +@InputRequirement(Requirement.INPUT_FORBIDDEN)
    +@SeeAlso({})
    +@CapabilityDescription(
    +        "Connects to the JMX RMI Url on the configured hostname and port.  "
    +                + "All domains are queried and can be filtered by providing the full domain name "
    +                + "and optional MBean type as whitelist or blacklist parameters.\n\n"
    +                + "Blacklist example to exclude all types that start with 'Memory' and GarbageCollector from the "
    +                + "java.lang domain and everything from the java.util.logging domain:"
    +                + "\n\njava.lang:Memory.* GarbageCollector,java.util.logging")
    +public class GetJMX extends AbstractProcessor {
    +
    +    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor
    --- End diff --
    
    properties must have displayName set 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by brianburnett <gi...@git.apache.org>.
Github user brianburnett commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    Ok - passed contrib-check and should be properly rebased.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1016: NIFI-2724 New JMX Processor

Posted by trixpan <gi...@git.apache.org>.
Github user trixpan commented on the issue:

    https://github.com/apache/nifi/pull/1016
  
    @brianburnett still not rebased correctly. My suggestion is to do again the trick of branching from the master
    
    git checkout master -b nifi-2724v3
    git cherry-pick your commit hash
    git cherry-pick your commit hash2
    ...
    git push - f origin nifi-2724



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---