Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

NIFI-5435 Prometheus /metrics http endpoint for monitoring integration #3257

Closed
wants to merge 11 commits into from

Conversation

SatwikBhandiwad
Copy link
Contributor

@SatwikBhandiwad SatwikBhandiwad commented Jan 10, 2019

NIFI-5435 Prometheus /metrics http endpoint for monitoring integration

In order to streamline the review of the contribution we ask you
to ensure the following steps have been taken:

For all changes:

  • Is there a JIRA ticket associated with this PR? Is it referenced
    in the commit message?

  • Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.

  • Has your PR been rebased against the latest commit within the target branch (typically master)?

  • Is your initial contribution a single, squashed commit?

For code changes:

  • Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
  • Have you written or updated unit tests to verify your changes?
  • If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under ASF 2.0?
  • If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
  • If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
  • If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?

For documentation related changes:

  • Have you ensured that format looks appropriate for the output in which it is rendered?

Note:

Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.

@SatwikBhandiwad SatwikBhandiwad changed the title Nifi 5435 NIFI-5435 Prometheus /metrics http endpoint for monitoring integration Jan 10, 2019
@usamaB
Copy link

usamaB commented Jan 22, 2019

hey @SatwikBhandiwad, Thanks for the work, looks nice.
I want to try this but before that, is it possible to add some custom metrics. Related to your business logic. e.g. send if connection queued count exceeds a threshold.
If yes, then how?

Thanks

@SatwikBhandiwad
Copy link
Contributor Author

hey @usamaB sorry for the late reply, in the latest patch I have included the code to expose counters provided by processors. Please have a look.

Copy link
Contributor

@pepov pepov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice feature overall, please see my comments inline!


@Tags({ "reporting", "prometheus", "metrics" })
@CapabilityDescription("")
@DefaultSchedule(strategy = SchedulingStrategy.TIMER_DRIVEN, period = "1 sec")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To stay safe I would say this would be something between 10sec and 1min by default instead of 1sec.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would say 1 min is the sanest default here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed it to 60 sec

@Override
public void onTrigger(final ReportingContext context) {

PrometheusServer.context = context;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What if multiple tasks are configured? I've just tested and it seems they will overwrite each other

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Made those variables private.


public static ReportingContext context;
public static boolean sendJvmMetrics;
public static String applicationId;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

these shouldnt be static, see above

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed them to private.

@pepov
Copy link
Contributor

pepov commented Feb 13, 2019

Also I tested the PR and I can confirm it's working (with the above mentioned caveats) on a two node cluster.

Copy link
Contributor

@kevdoran kevdoran left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the contribution @SatwikBhandiwad! A PrometheusReportingTask would definitely be useful.

Overall this looks like a nice start.

I think we would definitely want to add the ability to secure the endpoint, at least with TLS and TLS mutual auth, so the standard keystore/truststore properties needed to create an SSL context probably apply.

I had a few other high level comments for things that jumped out at me, along with what @pepov had which I agree with.

Thanks!

@@ -0,0 +1,46 @@
nifi-prometheus-nar
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At a glance it looks like this NOTICE file will need to be updated.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @kevdoran, working on SSL support for the endpoint.

PrometheusServer.logger = logger;
this.server = new Server(addr);

this.handler = new ServletContextHandler(server, "/metrics");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not a huge deal as this runs on its own port, but should this /metrics path be configurable?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.


static class MetricsServlet extends HttpServlet {
private CollectorRegistry nifiRegistry, jvmRegistry;
private ProcessGroupStatus rootGroupStatus;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like these class variables are only used in the doGet method below. Can they be local variables to that method?

import io.prometheus.client.CollectorRegistry;
import io.prometheus.client.Gauge;

public class PrometheusMetricsFactory {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Factory is a bit misleading class name. Perhaps PrometheusMetricsUtil would be better? If possible, I would add some unit tests for the results of the static methods as well.

<dependencies>
<dependency>
<groupId>org.glassfish.jersey.core</groupId>
<artifactId>jersey-client</artifactId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see jersey-client used in this extension unless I missed it. Is it brought in transitively by a dependency?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed this unused dependency.

HttpURLConnection con = (HttpURLConnection) url.openConnection();
con.setRequestMethod("GET");
int status = con.getResponseCode();
Assert.assertEquals(HttpURLConnection.HTTP_OK, status);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be nice to have some assertions on the results as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added assertion for result.

@SatwikBhandiwad
Copy link
Contributor Author

Thanks @pepov and @kevdoran for showing interest and reviewing the code. I'll address all the comments and come back soon.

Copy link
Contributor

@MikeThomsen MikeThomsen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall LGTM, but we need some cleanup. I'll try to find some time to dig into the L&N in depth.


private PrometheusServer prometheusServer;

static final PropertyDescriptor METRICS_ENDPOINT_PORT = new PropertyDescriptor.Builder().name("Prometheus Metrics Endpoint Port").description("The Port where prometheus metrics can be accessed")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the sake of readability, please reformat these property descriptors with one method call per line with appropriate indentation.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Formatted the code.

import org.eclipse.jetty.server.Server;

@Tags({ "reporting", "prometheus", "metrics" })
@CapabilityDescription("")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Need a few sentences in the capability description.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added a few lines. Please let me know if there are any changes required.


@Tags({ "reporting", "prometheus", "metrics" })
@CapabilityDescription("")
@DefaultSchedule(strategy = SchedulingStrategy.TIMER_DRIVEN, period = "1 sec")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would say 1 min is the sanest default here.


@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As a rule of thumb, we prefer that these be built up in a static constructor into a static list that is returned here so that we don't rebuild the list.


public static ReportingContext context;
public static boolean sendJvmMetrics;
public static String applicationId;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed.

private static final CollectorRegistry NIFI_REGISTRY = new CollectorRegistry();
private static final CollectorRegistry JVM_REGISTRY = new CollectorRegistry();

private static final Gauge AMOUNT_FLOWFILES_TOTAL = Gauge.build().name("process_group_amount_flowfiles_total").help("Total number of FlowFiles in ProcessGroup")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Builder methods should be on separate lines.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Formatted the code.

private Server server;
private ServletContextHandler handler;

public static ReportingContext context;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These should be private and accessed through getters and setters.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added getter and setter methods.


@Override
public void onTrigger(final ReportingContext context) {

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could this empty line be removed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed.

@MikeThomsen
Copy link
Contributor

@SatwikBhandiwad are you still tracking this?

@SatwikBhandiwad
Copy link
Contributor Author

Hi @MikeThomsen, I am working on the suggested changes.

@MikeThomsen
Copy link
Contributor

@SatwikBhandiwad ok, good. Ping me when you're ready.

Also, can you provide a comment that describes your test methodology for those of us new to Prometheus?

@SatwikBhandiwad
Copy link
Contributor Author

@MikeThomsen I just committed my code, why am I seeing 81 commits? Please help

@pepov
Copy link
Contributor

pepov commented Feb 25, 2019

I would suggest to get rid of the unnecessary merges and rebase your work on the current master.

First add apache remote and update it if you haven't already:

git remote add apache https://github.com/apache/nifi
git fetch apache

Then for example create a temporary branch from your current head

git checkout -b NIFI-5435-tmp NIFI-5435

then go back to NIFI-5435 and reset it to apache master

git checkout NIFI-5435
git reset apache/master --hard

Then you can cherry pick the relevant commits, without the merges from your tmp branch:

$  git log apache/master..NIFI-5435-tmp --no-merges --pretty=oneline --abbrev-commit
6b316416b NIFI-5435 Formated Code to adhere to NiFi checkstyle and Suggested code changes
e8deefa56 NIFI-5435 Added Counters exposed by processors
518fb6903 Update PrometheusMetricsFactory.java
fa9bb0e49 Update pom.xml
060bfb35a NIFI-5435 Adding Prometheus /metrics http endpoint
d19329063 NIFI-5435 Added Counters exposed by processors
b5f68a02c Update PrometheusMetricsFactory.java
8534288c8 Update pom.xml
7c1daf245 NIFI-5435 Adding Prometheus /metrics http endpoint

Or you can create a diff and add it as a single commit:

git apply <(git diff apache/master..NIFI-5435-tmp)
git add .
git commit -m ...

@SatwikBhandiwad
Copy link
Contributor Author

Thank you @pepov I'll try the above steps

@MikeThomsen
Copy link
Contributor

@SatwikBhandiwad for future reference, you don't need to pull the latest updates unless either GitHub is reporting that an automatic merge is impossible or we tell you to do so (ex. because there is a feature you need or you're so far behind you're referencing a previous release). If there's no automatic merge issue detected, as a rule of thumb we'll do the rebasing/squashing for you on approval and merge.

@pepov
Copy link
Contributor

pepov commented Feb 25, 2019

I think the problem is the default behaviour of pull in this case, that uses merge instead of rebase.

To properly update your commits I suggest using git fetch with git rebase or simply git pull --rebase. You can find a lot of content on why this is good (no merges), when to use (in local forks and private branches), and how it works:
https://www.google.com/search?q=git+pull+rebase+instead+of+merge

@SatwikBhandiwad
Copy link
Contributor Author

Hi @pepov I tried those above mentioned steps. I got
! [rejected] NIFI-5435 -> NIFI-5435 (non-fast-forward)
error: failed to push some refs to 'https://github.com/SatwikBhandiwad/nifi.git'
hint: Updates were rejected because the tip of your current branch is behind
hint: its remote counterpart. Integrate the remote changes (e.g.
hint: 'git pull ...') before pushing again.
hint: See the 'Note about fast-forwards' in 'git push --help' for details.

@SatwikBhandiwad
Copy link
Contributor Author

Is it fine if I create a new branch and create a new pull request by adding all reviewers and respective comments in the comments section?

@pepov
Copy link
Contributor

pepov commented Feb 26, 2019

you have to force push it using -f

@SatwikBhandiwad
Copy link
Contributor Author

@SatwikBhandiwad ok, good. Ping me when you're ready.

Also, can you provide a comment that describes your test methodology for those of us new to Prometheus?

Hi @MikeThomsen the test methodology involves providing the http endpoint of NiFi as target to Prometheus https://prometheus.io/docs/introduction/first_steps/#configuring-prometheus and checking whether the data is available in Prometheus https://prometheus.io/docs/guides/node-exporter/#exploring-node-exporter-metrics-through-the-prometheus-expression-browser.

For more info refer: https://prometheus.io/docs/introduction/first_steps/

@SatwikBhandiwad
Copy link
Contributor Author

Hi @MikeThomsen @kevdoran @pepov I've made the suggested changes. Please review the code and let me know if there are any changes required.

@kevdoran
Copy link
Contributor

kevdoran commented Mar 8, 2019

Thanks, @SatwikBhandiwad! I'll take a look at the latest on this branch when I get a chance

<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
<version>4.5.3</version>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this used outside of a test scope?

Copy link
Contributor

@kevdoran kevdoran left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @SatwikBhandiwad,

I finally had time to take another pass at this PR.

Thanks for making the security changes! As this is getting closer to a mergeable state, I looked at the changes in more detail than before. If you could look at my feedback and let me know what you think of those changes, that would be great. If you're able to push updates, I should be able to review them much faster as I have a good testing environment for running this branch now.

Thanks!

@SatwikBhandiwad
Copy link
Contributor Author

Hi @kevdoran thanks for reviewing . I am down with typhoid fever. I'll make the suggested changes after I recover.

@kevdoran
Copy link
Contributor

kevdoran commented Apr 7, 2019

No rush on this, @SatwikBhandiwad. Sorry to hear that, hope you feel better soon!

@SatwikBhandiwad
Copy link
Contributor Author

Hi @kevdoran I'll start working on the suggested changes

@MikeThomsen
Copy link
Contributor

@SatwikBhandiwad Any updates?

@SatwikBhandiwad
Copy link
Contributor Author

Hi @MikeThomsen , I will try to push the suggested changes by this weekend.

@SatwikBhandiwad
Copy link
Contributor Author

Hi @kevdoran @MikeThomsen . I've pushed the suggested code changes regarding the format of metrics exposed. Please review and let me know if there are any changes required.

@MikeThomsen
Copy link
Contributor

@SatwikBhandiwad I'll try to get a review done in the next day or so.

@SatwikBhandiwad
Copy link
Contributor Author

SatwikBhandiwad commented May 27, 2019

I've made all the code changes except NOTICE file. I need some help with that. Please check my comments here: #3257 (comment)

@MikeThomsen
Copy link
Contributor

@SatwikBhandiwad can this reporting task break out the stats on a per-PG basis? I have Prometheus and NiFi synced up and can see the stats for the flow as a whole.

@SatwikBhandiwad
Copy link
Contributor Author

@MikeThomsen right now only root PG metrics are reported. Do you want something like MetricsReportingTask where if a PG id is given, metrics related to that PG are reported else root PG metrics are reported ?

@MikeThomsen
Copy link
Contributor

Yeah, I think we need something like that. Though allowing users to enter the PG name and then lookup the ID would be more intuitive.

@kevdoran
Copy link
Contributor

Per-PG metrics, as a dimension, would be a great enhancement. Letting users enter a PG name, but internally using the PG id, is a good idea, but PG names are not guaranteed to be unique, right? So what should the mechanic be in that case?

I plan to try and pull this down and test it sometime this week. Thanks for the work on this @SatwikBhandiwad!

@MikeThomsen
Copy link
Contributor

They're not guaranteed to be unique, but I think that's a case where we have two choices:

  1. Track both and maybe add an arbitrary index to them like PGNAME_1
  2. Just throw an error.

rootGroupStatus = PrometheusServer.this.context.getEventAccess().getControllerStatus();
ServletOutputStream response = resp.getOutputStream();
OutputStreamWriter osw = new OutputStreamWriter(response);
nifiRegistry = PrometheusMetricsUtil.createNifiMetrics(rootGroupStatus, PrometheusServer.this.instanceId);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This part is reporting the "rollup" statistics for the entire canvas (i.e. root process group). I think it would be really powerful if we could report metrics (aka "status" in the existing NiFi Reporting Task jargon) for each component in the system, like the SiteToSiteStatusReportingTask does. That way we could zero in on only connections for example, to find backpressure, hotspots, etc. Having said that, I can also see the benefit of just having the rollup metrics, they're faster to query and perhaps offer enough information themselves to be useful for querying, dashboarding etc.

How about we finish the review of this PR with the current functionality, then add a follow-on Jira to handle per-component (or per-PG or both, configurable of course) metrics? Perhaps we could make this a property of the reporting task, something like "Status Reporting Level" defaulting to "Root Process Group", then we could add another option later for "All Process Groups" and/or "All Components". I'd be happy to take on the follow-on Jira and continue the discussion there. Thoughts?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good ideas, thanks @mattyb149.

+1 from me in favor of getting this merged with the current scope of functionality (root PG roll up) and then adding additional granularity (per-pg, and per-component) as a follow-on task.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think makes sense. We'll just need to document this component as an experimental one if those changes don't make it before 1.10

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure we'd need to mark it Experimental unless we anticipate it being refactored such that the original capability will not be retained. This PR seems like Minimum Viable Product to me, a good starting point for capturing metrics for the NiFi instance.

Having said that, I'm sure there will be a good deal of refactor under the hood when we add the options to collect at the PG or component level, but I think we can maintain behavior (whether it's added by 1.10 or not) by having the dropdown with only the current option available until the others are implemented. Just my opinion though, I'm happy to go with the will of the group.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @mattyb149 for reviewing. I've made the suugested changes by adding process_group_name and process_group_id labels to metrics , also changed logger to debug level.


@Override
protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) throws ServletException, IOException {
logger.info("PrometheusServer Do get called");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be at the debug or trace level

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And wrapped with something like if (logger.isDebugEnabled())

private static final Gauge AMOUNT_FLOWFILES_SENT = Gauge.build()
.name("nifi_process_group_amount_flowfiles_sent")
.help("Total number of FlowFiles in ProcessGroup sent")
.labelNames("instance", "process_group")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add process_group_name as a label? It helps when querying and dashboarding, even if you have non-unique names for processors (as you can always use the process_group ID as another filter/facet). Speaking of which, can we change the label process_group to process_group_id, to be more specific about its contents?

@MikeThomsen
Copy link
Contributor

@mattyb149 I agree that it's a MVP based on what I've seen. I'm going to try to take another look through the PR and make sure it's all good with particular focus on L&N. Then I'll merge unless some tells me to hold up.

@mattyb149
Copy link
Contributor

I had a couple of other minor comments I’d like to see addressed, with logging levels, adding the processor group name, and renaming the ID label, after that I’m a +1

@MikeThomsen
Copy link
Contributor

Ok. I think L&N is still broken here.

Copy link
Contributor

@MikeThomsen MikeThomsen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@SatwikBhandiwad I think you're very close to MVP. Added some examples to help you close out the L&N quickly.

http://creativecommons.org/publicdomain/zero/1.0/

************************
Common Development and Distribution License 1.1
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see any of these listed with mvn dependency:tree so they can be removed.


************************
Common Development and Distribution License 1.1
************************
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add this under the Apache segment:

    (ASLv2) Prometheus Simple Client libraries
        Copyright 2012-2019 The Prometheus Authors

I took that from their NOTICE file.

Then add:

  (ASLv2) Jetty
    The following NOTICE information applies:
       Jetty Web Container
       Copyright 1995-2017 Mort Bay Consulting Pty Ltd.

  (ASLv2) Apache Commons Codec
      The following NOTICE information applies:
        Apache Commons Codec
        Copyright 2002-2014 The Apache Software Foundation

        src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
        contains test data from http://aspell.net/test/orig/batch0.tab.
        Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)

        ===============================================================================

        The content of package org.apache.commons.codec.language.bm has been translated
        from the original php source code available at http://stevemorse.org/phoneticinfo.htm
        with permission from the original authors.
        Original source copyright:
        Copyright (c) 2008 Alexander Beider & Stephen P. Morse.

    (ASLv2) Apache Commons Lang
    The following NOTICE information applies:
      Apache Commons Lang
      Copyright 2001-2017 The Apache Software Foundation

      This product includes software from the Spring Framework,
      under the Apache License 2.0 (see: StringUtils.containsWhitespace())

Then add a new MIT section like this:

************
MIT
************

  	This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
  	a temporary self-signed X.509 certificate when the JVM does not provide the
  	equivalent functionality.  It can be obtained at:

  	  * LICENSE:
  	    * license/LICENSE.bouncycastle.txt (MIT License)
  	  * HOMEPAGE:
  	    * http://www.bouncycastle.org/

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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a section for jBCrypt following the examples we already have in our code like this:

This product bundles 'jBCrypt' which is available under an MIT license.
For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE

    Copyright (c) 2006 Damien Miller <djm@mindrot.org>

    Permission to use, copy, modify, and distribute this software for any
    purpose with or without fee is hereby granted, provided that the above
    copyright notice and this permission notice appear in all copies.

    THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
    WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
    MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
    ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
    WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
    ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
    OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.

import org.apache.nifi.ssl.SSLContextService;
import org.eclipse.jetty.server.Server;

@Tags({ "reporting", "prometheus", "metrics" })
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would be good to add something about it being a time series data set here.


@Tags({ "reporting", "prometheus", "metrics" })
@CapabilityDescription("Reports metrics in Prometheus format by creating /metrics http endpoint which can be used for external monitoring of the application."
+ "The reporting task reports a set of metrics regarding the JVM (optional) and the NiFi instance")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Need a space after the period on the previous line or at the start of the second sentence.


@Override
protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) throws ServletException, IOException {
logger.info("PrometheusServer Do get called");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And wrapped with something like if (logger.isDebugEnabled())

@SatwikBhandiwad
Copy link
Contributor Author

@MikeThomsen I've pushed the suggested changes. Please review and let me know if any changes are required. I don't know why a few travis builds have failed

Copy link
Contributor

@MikeThomsen MikeThomsen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kevdoran @mattyb149 looks like all of the changes are in, particularly on the L&N.

@asfgit asfgit closed this in 8560eb9 May 30, 2019
@SatwikBhandiwad
Copy link
Contributor Author

Thanks @pepov @kevdoran @MikeThomsen @mattyb149 for all your support and guidance in my first open source contribution, hope to contribute more in future.

@kevdoran
Copy link
Contributor

Nice work, @SatwikBhandiwad! Thanks for being willing to make those changes. Welcome to the Apache NiFi community!

@MikeThomsen
Copy link
Contributor

@SatwikBhandiwad what @kevdoran said. You did a great job on taking responsibility for seeing this through.

@usamaB
Copy link

usamaB commented Oct 29, 2019

Hi guys it was working for me months ago. We updated the NIFI and somehow my code got lost. I didnt had the correct version on Git and now I wrote the code again. But I'm always getting Jetty Bind Exception already in use.
How can this be resolved I have tried giving different ports its always the same.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
7 participants